[
  {
    "path": ".asf.yaml",
    "content": "# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License. You may obtain a copy of the License at\n#\n# http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n\n# https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features\n---\ngithub:\n  description: \"Apache Spark Connect Client for Golang\"\n  homepage: https://spark.apache.org/\n  enabled_merge_buttons:\n    merge: false\n    squash: true\n    rebase: true\n  features:\n    # Enable the \"Issues\" tab\n    issues: true\n    # Enable the \"Projects\" tab\n    projects: true\n\nnotifications:\n  pullrequests: reviews@spark.apache.org\n  issues: reviews@spark.apache.org\n  commits: commits@spark.apache.org\n\n"
  },
  {
    "path": ".github/PULL_REQUEST_TEMPLATE",
    "content": "<!--\nThanks for sending a pull request!  Here are some tips for you:\n  1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html\n  2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html\n  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.\n  4. Be sure to keep the PR description updated to reflect all changes.\n  5. Please write your PR title to summarize what this PR proposes.\n  6. If possible, provide a concise example to reproduce the issue for a faster review.\n  7. If you want to add a new configuration, please read the guideline first for naming configurations in\n     'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.\n  8. If you want to add or modify an error type or message, please read the guideline first in\n     'core/src/main/resources/error/README.md'.\n-->\n\n### What changes were proposed in this pull request?\n<!--\nPlease clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.\nIf possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.\n  1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.\n  2. If you fix some SQL features, you can provide some references of other DBMSes.\n  3. If there is design documentation, please add the link.\n  4. If there is a discussion in the mailing list, please add the link.\n-->\n\n\n### Why are the changes needed?\n<!--\nPlease clarify why the changes are needed. For instance,\n  1. If you propose a new API, clarify the use case for a new API.\n  2. If you fix a bug, you can clarify why it is a bug.\n-->\n\n\n### Does this PR introduce _any_ user-facing change?\n<!--\nNote that it means *any* user-facing change including all aspects such as the documentation fix.\nIf yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.\nIf possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.\nIf no, write 'No'.\n-->\n\n\n### How was this patch tested?\n<!--\nIf tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.\nIf it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.\nIf tests were not added, please describe why they were not added and/or why it was difficult to add.\nIf benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.\n-->\n"
  },
  {
    "path": ".github/dependabot.yml",
    "content": "# To get started with Dependabot version updates, you'll need to specify which\n# package ecosystems to update and where the package manifests are located.\n# Please see the documentation for all configuration options:\n# https://docs.github.com/code-security/dependabot/dependabot-version-updates/configuration-options-for-the-dependabot.yml-file\n\nversion: 2\nupdates:\n  - package-ecosystem: \"gomod\" # See documentation for possible values\n    directory: \"/\" # Location of package manifests\n    schedule:\n      interval: \"weekly\"\n"
  },
  {
    "path": ".github/workflows/build.yml",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements.  See the NOTICE file\n# distributed with this work for additional information\n# regarding copyright ownership.  The ASF licenses this file\n# to you under the Apache License, Version 2.0 (the\n# \"License\"); you may not use this file except in compliance\n# with the License.  You may obtain a copy of the License at\n#\n#   http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing,\n# software distributed under the License is distributed on an\n# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n# KIND, either express or implied.  See the License for the\n# specific language governing permissions and limitations\n# under the License.\n#\n\n# Intentionally has a general name.\n# because the test status check created in GitHub Actions\n# currently randomly picks any associated workflow.\n# So, the name was changed to make sense in that context too.\n# See also https://github.community/t/specify-check-suite-when-creating-a-checkrun/118380/10\n\nname: \"On pull requests\"\non:\n  workflow_dispatch:\n  pull_request:\n  push:\n    branches:\n      - master\n\nenv:\n  SPARK_VERSION: '4.0.0'\n  HADOOP_VERSION: '3'\n\npermissions:\n  # Required: allow read access to the content for analysis.\n  contents: read\n  # Optional: allow read access to pull request. Use with `only-new-issues` option.\n  pull-requests: read\n  # Optional: allow write access to checks to allow the action to annotate code in the PR.\n  checks: write\n\n\njobs:\n  build:\n    name: Build & Test Client\n    runs-on: ubuntu-latest\n    steps:\n      - name: Checkout Repository\n        uses: actions/checkout@v4\n        with:\n          submodules: recursive\n      - uses: actions/setup-go@v5\n        name: Setup Go\n        with:\n          go-version-file: 'go.mod'\n      - uses: actions/setup-python@v5\n        with:\n          python-version: '3.10'\n\n      - uses: actions/setup-java@v4\n        with:\n          java-version: '17'\n          distribution: zulu\n\n      - name: Cache Spark Installation\n        uses: actions/cache@v4\n        id: cache\n        with:\n          key: v2-spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n          path: |\n            /home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n\n      - name: Setup SBT\n        uses: sbt/setup-sbt@v1\n\n      - name: Setup Apache Spark\n        if: steps.cache.outputs.cache-hit != 'true'\n        run: |\n          set -x\n          echo \"Apache Spark is not installed\"\n          # Access the directory.\n          mkdir -p ~/deps/\n          wget -q https://dlcdn.apache.org/spark/spark-${{ env.SPARK_VERSION }}/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz\n          tar -xzf spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz -C ~/deps/\n          # Delete the old file\n          rm spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz\n\n          ls -lah ~/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n          du -hs ~/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n\n          # Setup the Environment Variables\n          echo \"Apache Spark is ready to use\"\n          echo \"SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\" >> \"$GITHUB_ENV\"\n      - name: Run Build & Test\n        run: |\n          go mod download -x\n          # Remove dependency on gen until Spark 4 has the fix for the pipelines.proto\n          # make gen\n          make\n          make test\n      - name: Run Example Spark Submit Application\n        run: |\n          export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n          make\n          cd java\n          sbt publishLocal\n          ./run.sh ../cmd/spark-connect-example-spark-session/spark-connect-example-spark-session\n      - name: Run Integration Test\n        run: |\n          export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n          # Remove dependency on gen until Spark 4 has the fix for the pipelines.proto\n          # make gen \n          make && make integration\n      - name: Run Code Coverage\n        run: |\n          export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}\n          make coverage\n# Disable the code coverage check for now until https://github.com/PaloAltoNetworks/cov/issues/8 is fixed.\n#      - uses: PaloAltoNetworks/cov@3.0.0\n#        with:\n#          cov_mode: coverage\n#          main_branch: master\n#          cov_threshold: 60\n      - name: golangci-lint\n        uses: golangci/golangci-lint-action@v8\n        with:\n          version: v2.1\n\n"
  },
  {
    "path": ".gitignore",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\n\n# All generated files\ninternal/generated.out\n\n# Ignore Coverage Files\ncoverage*\ncov.report\n\n# Ignore IDE files\n.idea/\n\n# Ignore binaries\ncmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client\ncmd/spark-connect-example-spark-session/spark-connect-example-spark-session\n\ntarget\nlib\n\ndeps\n\n.DS_Store"
  },
  {
    "path": ".gitmodules",
    "content": "[submodule \"spark\"]\n\tpath = sparksrc\n\turl = https://github.com/apache/spark.git\n[submodule \"sparksrc\"]\n\tbranch = branch-4.0\n"
  },
  {
    "path": ".golangci.yml",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\nversion: \"2\"\nlinters:\n  exclusions:\n    generated: lax\n    presets:\n      - comments\n      - common-false-positives\n      - legacy\n      - std-error-handling\n    paths:\n      - third_party$\n      - builtin$\n      - examples$\nformatters:\n  enable:\n    - gofumpt\n  exclusions:\n    generated: lax\n    paths:\n      - third_party$\n      - builtin$\n      - examples$\n"
  },
  {
    "path": "CONTRIBUTING.md",
    "content": "## Contributing to Spark\n\n*Before opening a pull request*, review the\n[Contributing to Spark guide](https://spark.apache.org/contributing.html).\nIt lists steps that are required before creating a PR. In particular, consider:\n\n- Is the change important and ready enough to ask the community to spend time reviewing?\n- Have you searched for existing, related JIRAs and pull requests?\n- Is this a new feature that can stand alone as a [third party project](https://spark.apache.org/third-party-projects.html) ?\n- Is the change being proposed clearly explained and motivated?\n\nWhen you contribute code, you affirm that the contribution is your original work and that you\nlicense the work to the project under the project's open source license. Whether or not you\nstate this explicitly, by submitting any copyrighted material via pull request, email, or\nother means you agree to license the material under the project's open source license and\nwarrant that you have the legal authority to do so.\n\n\n### Code Style and Checks\n\nWhen submitting code we use a number of checks in our continous integration system to ensure\na consitent style and adherence to license rules. You can run these checks locally by running:\n\n```bash\nmake check\n```\n\nThis requires the following tools to be present in your `PATH`:\n\n1. Java for checking license headers\n2. [gofumpt](https://github.com/mvdan/gofumpt) for formatting Go code\n3. [golangci-lint](https://golangci-lint.run/) for linting Go code\n\n### Running Tests\n\nTo run the tests locally, you can run:\n\n```bash\nmake test\n```\n\nThis will run the unit tests. If you want to run the integration tests, you can run (you\nneed to set environment variable `SPARK_HOME` pointing to existing directory with unpacked\nApache Spark 3.5+ distribution):\n\n```bash\nmake integration\n```\n\nLastly, if you want to run all tests (unit and integration) and generate the coverage\nanalysis, you can run:\n\n```bash\nmake fulltest\n```\n\nThe output of the coverage analysis will be in the `coverage.out` file. An HTML version of\nthe coverage report is generated and accessible at `coverage.html`.\n\n### How to write tests\n\nPlease make sure that you have proper testing for the new code your adding. As part of the\ncode base we started to add mocks that allow you to simulate a lot of the necessary API\nand don't require a running Spark instance.\n\n`mock.ProtoClient` is a mock implementation of the `SparkConnectService_ExecutePlanClient`\ninterface which is the server-side stream of messages coming as a response from the server.\n\n`testutils.NewConnectServiceClientMock` will create a mock client that implements the\n`SparkConnectServiceClient` interface.\n\nThe combination of these two mocks allows you to test the client side of the code without\nhaving to connect to Spark.\n\n### What to contribute\n\nWe welcome contributions of all kinds to the `spark-connect-go` project. Some examples of\ncontributions are providing implementations of functionality that is missing in the Go\nimplementation. Some examples are, but are not limited to:\n\n* Adding an existing feature of the DataFrame API in Golang.\n* Adding support for a builtin function in the Spark API in Golang.\n* Improving error handling in the client.\n\nIf you are unsure about whether a contribution is a good fit, feel free to open an issue\nin the Apache Spark Jira.\n"
  },
  {
    "path": "LICENSE",
    "content": "                               Apache License\n                           Version 2.0, January 2004\n                        http://www.apache.org/licenses/\n\n   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION\n\n   1. Definitions.\n\n      \"License\" shall mean the terms and conditions for use, reproduction,\n      and distribution as defined by Sections 1 through 9 of this document.\n\n      \"Licensor\" shall mean the copyright owner or entity authorized by\n      the copyright owner that is granting the License.\n\n      \"Legal Entity\" shall mean the union of the acting entity and all\n      other entities that control, are controlled by, or are under common\n      control with that entity. For the purposes of this definition,\n      \"control\" means (i) the power, direct or indirect, to cause the\n      direction or management of such entity, whether by contract or\n      otherwise, or (ii) ownership of fifty percent (50%) or more of the\n      outstanding shares, or (iii) beneficial ownership of such entity.\n\n      \"You\" (or \"Your\") shall mean an individual or Legal Entity\n      exercising permissions granted by this License.\n\n      \"Source\" form shall mean the preferred form for making modifications,\n      including but not limited to software source code, documentation\n      source, and configuration files.\n\n      \"Object\" form shall mean any form resulting from mechanical\n      transformation or translation of a Source form, including but\n      not limited to compiled object code, generated documentation,\n      and conversions to other media types.\n\n      \"Work\" shall mean the work of authorship, whether in Source or\n      Object form, made available under the License, as indicated by a\n      copyright notice that is included in or attached to the work\n      (an example is provided in the Appendix below).\n\n      \"Derivative Works\" shall mean any work, whether in Source or Object\n      form, that is based on (or derived from) the Work and for which the\n      editorial revisions, annotations, elaborations, or other modifications\n      represent, as a whole, an original work of authorship. For the purposes\n      of this License, Derivative Works shall not include works that remain\n      separable from, or merely link (or bind by name) to the interfaces of,\n      the Work and Derivative Works thereof.\n\n      \"Contribution\" shall mean any work of authorship, including\n      the original version of the Work and any modifications or additions\n      to that Work or Derivative Works thereof, that is intentionally\n      submitted to Licensor for inclusion in the Work by the copyright owner\n      or by an individual or Legal Entity authorized to submit on behalf of\n      the copyright owner. For the purposes of this definition, \"submitted\"\n      means any form of electronic, verbal, or written communication sent\n      to the Licensor or its representatives, including but not limited to\n      communication on electronic mailing lists, source code control systems,\n      and issue tracking systems that are managed by, or on behalf of, the\n      Licensor for the purpose of discussing and improving the Work, but\n      excluding communication that is conspicuously marked or otherwise\n      designated in writing by the copyright owner as \"Not a Contribution.\"\n\n      \"Contributor\" shall mean Licensor and any individual or Legal Entity\n      on behalf of whom a Contribution has been received by Licensor and\n      subsequently incorporated within the Work.\n\n   2. Grant of Copyright License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      copyright license to reproduce, prepare Derivative Works of,\n      publicly display, publicly perform, sublicense, and distribute the\n      Work and such Derivative Works in Source or Object form.\n\n   3. Grant of Patent License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      (except as stated in this section) patent license to make, have made,\n      use, offer to sell, sell, import, and otherwise transfer the Work,\n      where such license applies only to those patent claims licensable\n      by such Contributor that are necessarily infringed by their\n      Contribution(s) alone or by combination of their Contribution(s)\n      with the Work to which such Contribution(s) was submitted. If You\n      institute patent litigation against any entity (including a\n      cross-claim or counterclaim in a lawsuit) alleging that the Work\n      or a Contribution incorporated within the Work constitutes direct\n      or contributory patent infringement, then any patent licenses\n      granted to You under this License for that Work shall terminate\n      as of the date such litigation is filed.\n\n   4. Redistribution. You may reproduce and distribute copies of the\n      Work or Derivative Works thereof in any medium, with or without\n      modifications, and in Source or Object form, provided that You\n      meet the following conditions:\n\n      (a) You must give any other recipients of the Work or\n          Derivative Works a copy of this License; and\n\n      (b) You must cause any modified files to carry prominent notices\n          stating that You changed the files; and\n\n      (c) You must retain, in the Source form of any Derivative Works\n          that You distribute, all copyright, patent, trademark, and\n          attribution notices from the Source form of the Work,\n          excluding those notices that do not pertain to any part of\n          the Derivative Works; and\n\n      (d) If the Work includes a \"NOTICE\" text file as part of its\n          distribution, then any Derivative Works that You distribute must\n          include a readable copy of the attribution notices contained\n          within such NOTICE file, excluding those notices that do not\n          pertain to any part of the Derivative Works, in at least one\n          of the following places: within a NOTICE text file distributed\n          as part of the Derivative Works; within the Source form or\n          documentation, if provided along with the Derivative Works; or,\n          within a display generated by the Derivative Works, if and\n          wherever such third-party notices normally appear. The contents\n          of the NOTICE file are for informational purposes only and\n          do not modify the License. You may add Your own attribution\n          notices within Derivative Works that You distribute, alongside\n          or as an addendum to the NOTICE text from the Work, provided\n          that such additional attribution notices cannot be construed\n          as modifying the License.\n\n      You may add Your own copyright statement to Your modifications and\n      may provide additional or different license terms and conditions\n      for use, reproduction, or distribution of Your modifications, or\n      for any such Derivative Works as a whole, provided Your use,\n      reproduction, and distribution of the Work otherwise complies with\n      the conditions stated in this License.\n\n   5. Submission of Contributions. Unless You explicitly state otherwise,\n      any Contribution intentionally submitted for inclusion in the Work\n      by You to the Licensor shall be under the terms and conditions of\n      this License, without any additional terms or conditions.\n      Notwithstanding the above, nothing herein shall supersede or modify\n      the terms of any separate license agreement you may have executed\n      with Licensor regarding such Contributions.\n\n   6. Trademarks. This License does not grant permission to use the trade\n      names, trademarks, service marks, or product names of the Licensor,\n      except as required for reasonable and customary use in describing the\n      origin of the Work and reproducing the content of the NOTICE file.\n\n   7. Disclaimer of Warranty. Unless required by applicable law or\n      agreed to in writing, Licensor provides the Work (and each\n      Contributor provides its Contributions) on an \"AS IS\" BASIS,\n      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or\n      implied, including, without limitation, any warranties or conditions\n      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A\n      PARTICULAR PURPOSE. You are solely responsible for determining the\n      appropriateness of using or redistributing the Work and assume any\n      risks associated with Your exercise of permissions under this License.\n\n   8. Limitation of Liability. In no event and under no legal theory,\n      whether in tort (including negligence), contract, or otherwise,\n      unless required by applicable law (such as deliberate and grossly\n      negligent acts) or agreed to in writing, shall any Contributor be\n      liable to You for damages, including any direct, indirect, special,\n      incidental, or consequential damages of any character arising as a\n      result of this License or out of the use or inability to use the\n      Work (including but not limited to damages for loss of goodwill,\n      work stoppage, computer failure or malfunction, or any and all\n      other commercial damages or losses), even if such Contributor\n      has been advised of the possibility of such damages.\n\n   9. Accepting Warranty or Additional Liability. While redistributing\n      the Work or Derivative Works thereof, You may choose to offer,\n      and charge a fee for, acceptance of support, warranty, indemnity,\n      or other liability obligations and/or rights consistent with this\n      License. However, in accepting such obligations, You may act only\n      on Your own behalf and on Your sole responsibility, not on behalf\n      of any other Contributor, and only if You agree to indemnify,\n      defend, and hold each Contributor harmless for any liability\n      incurred by, or claims asserted against, such Contributor by reason\n      of your accepting any such warranty or additional liability.\n\n   END OF TERMS AND CONDITIONS\n\n   APPENDIX: How to apply the Apache License to your work.\n\n      To apply the Apache License to your work, attach the following\n      boilerplate notice, with the fields enclosed by brackets \"[]\"\n      replaced with your own identifying information. (Don't include\n      the brackets!)  The text should be enclosed in the appropriate\n      comment syntax for the file format. We also recommend that a\n      file or class name and description of purpose be included on the\n      same \"printed page\" as the copyright notice for easier\n      identification within third-party archives.\n\n   Copyright [yyyy] [name of copyright owner]\n\n   Licensed under the Apache License, Version 2.0 (the \"License\");\n   you may not use this file except in compliance with the License.\n   You may obtain a copy of the License at\n\n       http://www.apache.org/licenses/LICENSE-2.0\n\n   Unless required by applicable law or agreed to in writing, software\n   distributed under the License is distributed on an \"AS IS\" BASIS,\n   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n   See the License for the specific language governing permissions and\n   limitations under the License.\n\n\n"
  },
  {
    "path": "Makefile",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\n\nFIRST_GOPATH              := $(firstword $(subst :, ,$(GOPATH)))\nPKGS                      := $(shell go list ./... | grep -v /tests | grep -v /xcpb | grep -v /gpb | grep -v /generated)\nGOFILES_NOVENDOR          := $(shell find . -name vendor -prune -o -type f -name '*.go' -not -name '*.pb.go' -print)\nGOFILES_BUILD             := $(shell find . -type f -name '*.go' -not -name '*_test.go')\nPROTOFILES                := $(shell find . -name vendor -prune -o -type f -name '*.proto' -print)\n\nALLGOFILES\t\t\t\t  \t\t\t:= $(shell find . -type f -name '*.go' -not -name '*.pb.go')\nDATE                      := $(shell date -u -d \"@$(SOURCE_DATE_EPOCH)\" '+%FT%T%z' 2>/dev/null || date -u '+%FT%T%z')\n\nBUILDFLAGS_NOPIE\t\t  :=\nBUILDFLAGS                ?= $(BUILDFLAGS_NOPIE) -buildmode=pie\nTESTFLAGS                 ?=\nPWD                       := $(shell pwd)\nPREFIX                    ?= $(GOPATH)\nBINDIR                    ?= $(PREFIX)/bin\nGO                        := go\nGOOS                      ?= $(shell go version | cut -d' ' -f4 | cut -d'/' -f1)\nGOARCH                    ?= $(shell go version | cut -d' ' -f4 | cut -d'/' -f2)\nTAGS                      ?= netgo\nSHELL = bash\nGOFUMPT_SPLIT_LONG_LINES  := on\n\n## Build tools\nBUF                       := $(GO) run github.com/bufbuild/buf/cmd/buf@v1.26.1\n\nBINARIES\t\t\t\t  := cmd/spark-connect-example-spark-session/spark-connect-example-spark-session cmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client\n\n# Define the location of SPARK_HOME because we need that to depend on the build paths\nMAKEFILE_DIR:=$(shell dirname $(realpath $(firstword $(MAKEFILE_LIST))))\n\nPROTO_SRC = $(shell find internal/generated -type f -name *.proto )\n\n\nOK := $(shell tput setaf 6; echo ' [OK]'; tput sgr0;)\n\nall: build\n\nbuild: $(BUILD_OUTPUT) $(BINARIES)\n\ncmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client: $(GOFILES_BUILD)\n\t@echo \">> BUILD, output = $@\"\n\t@cd $(dir $@) && $(GO) build -o $(notdir $@) $(BUILDFLAGS)\n\t@printf '%s\\n' '$(OK)'\n\ncmd/spark-connect-example-spark-session/spark-connect-example-spark-session: $(GOFILES_BUILD)\n\t@echo \">> BUILD, output = $@\"\n\t@cd $(dir $@) && $(GO) build -o $(notdir $@) $(BUILDFLAGS)\n\t@printf '%s\\n' '$(OK)'\n\ninternal/generated.out:\n\t@echo -n \">> BUILD, output = $@\"\n\t$(BUF) generate --debug -vvv\n\t@touch internal/generated.out\n\t@printf '%s\\n' '$(OK)'\n\ngen: internal/generated.out\n\n$(GOFILES_BUILD):\n\n$(BUILD_OUTPUT): $(GOFILES_BUILD)\n\t@echo -n \">> BUILD, output = $@\"\n\t@$(GO) build -o $@ $(BUILDFLAGS)\n\t@printf '%s\\n' '$(OK)'\n\nlint: $(BUILD_OUTPUT)\n\t@golangci-lint run\n\nfmt:\n\t@echo -n \">> glongci-lint: fix\"\n\tenv GOFUMPT_SPLIT_LONG_LINES=$(GOFUMPT_SPLIT_LONG_LINES) golangci-lint run --fix\n\ntest: $(BUILD_OUTPUT)\n\t@echo \">> TEST, \\\"verbose\\\"\"\n\t@$(foreach pkg, $(PKGS),\\\n\t    @echo -n \"     \";\\\n\t\t$(GO) test -v -run '(Test|Example)' $(BUILDFLAGS) $(TESTFLAGS) $(pkg) || exit 1)\n\ncoverage: $(BUILD_OUTPUT)\n\t@echo \">> TEST, \\\"coverage\\\"\"\n\t@$(GO) test -cover -coverprofile=coverage.out -covermode=atomic -coverpkg=./spark/...,./internal/tests/... ./spark/... ./internal/tests/...\n\t@$(GO) tool cover -html=coverage.out -o coverage.html\n\nintegration: $(BUILD_OUTPUT)\n\t@echo \">> TEST, \\\"integration\\\"\"\n\t@$(GO) test ./internal/tests/...\n\ncheck:\n\t@echo -n \">> CHECK\"\n\t./dev/check-license\n\t@echo -n \">> glongci-lint: \"\n\tenv GOFUMPT_SPLIT_LONG_LINES=$(GOFUMPT_SPLIT_LONG_LINES) golangci-lint run\n\nclean:\n\t@echo -n \">> CLEAN\"\n\t@$(GO) clean -i ./...\n\t@rm -f ./coverage-all.html\n\t@rm -f ./coverage-all.out\n\t@rm -f ./coverage.out\n\t@find . -type f -name \"coverage.out\" -delete\n\t@printf '%s\\n' '$(OK)'\n\ncleangen:\n\t@rm -rf ./internal/generated\n\t@rm  -f ./internal/generated.out\n\ncleanall: clean cleangen"
  },
  {
    "path": "NOTICE",
    "content": "Apache Spark\nCopyright 2014 and onwards The Apache Software Foundation.\n\nThis product includes software developed at\nThe Apache Software Foundation (http://www.apache.org/)."
  },
  {
    "path": "README.md",
    "content": "# Apache Spark Connect Client for Golang\n\nThis project houses the **experimental** client for [Spark\nConnect](https://spark.apache.org/docs/latest/spark-connect-overview.html) for\n[Apache Spark](https://spark.apache.org/) written in [Golang](https://go.dev/).\n\n## Current State of the Project\n\nCurrently, the Spark Connect client for Golang is highly experimental and should\nnot be used in any production setting. In addition, the PMC of the Apache Spark\nproject reserves the right to withdraw and abandon the development of this project\nif it is not sustainable.\n\n## Getting started\n\nThis section explains how to run Spark Connect Go locally.\n\nStep 1: Install Golang: https://go.dev/doc/install.\n\nStep 2: Ensure you have installed `buf CLI` installed, [more info here](https://buf.build/docs/installation/)\n\nStep 3: Run the following commands to setup the Spark Connect client.\n\nBuilding with Spark in case you need to re-generate the source files from the proto sources.\n\n```\ngit clone https://github.com/apache/spark-connect-go.git\ngit submodule update --init --recursive\n\nmake gen && make test\n\n```\n\nBuilding without Spark\n\n```\ngit clone https://github.com/apache/spark-connect-go.git\nmake && make test\n```\n\nStep 4: Setup the Spark Driver on localhost.\n\n1. [Download Spark distribution](https://spark.apache.org/downloads.html) (4.0.0+), unzip the package.\n\n2. Start the Spark Connect server with the following command (make sure to use a package version that matches your Spark distribution):\n\n```\nsbin/start-connect-server.sh\n```\n\nStep 5: Run the example Go application.\n\n```\ngo run cmd/spark-connect-example-spark-session/main.go\n```\n\n## Runnning Spark Connect Go Application in a Spark Cluster\n\nTo run the Spark Connect Go application in a Spark Cluster, you need to build the Go application and submit it to the Spark Cluster. You can find a more detailed example runner and wrapper script in the `java` directory.\n\nSee the guide here: [Sample Spark-Submit Wrapper](java/README.md).\n\n## How to write Spark Connect Go Application in your own project\n\nSee [Quick Start Guide](quick-start.md)\n\n## High Level Design\n\nThe overall goal of the design is to find a good balance of principle of the least surprise for\ndeveloeprs that are familiar with the APIs of Apache Spark and idiomatic Go usage. The high-level\nstructure of the packages follows roughly the PySpark giudance but with Go idioms.\n\n## Contributing\n\nPlease review the [Contribution to Spark guide](https://spark.apache.org/contributing.html)\nfor information on how to get started contributing to the project.\n"
  },
  {
    "path": "buf.gen.yaml",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\nversion: v1\nplugins:\n  - plugin: buf.build/protocolbuffers/go:v1.30.0\n    out: .\n  - plugin: buf.build/grpc/go:v1.3.0\n    out: .\n\n"
  },
  {
    "path": "buf.work.yaml",
    "content": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\nversion: v1\ndirectories:\n  - sparksrc/sql/connect/common/src/main/protobuf\n"
  },
  {
    "path": "cmd/spark-connect-example-raw-grpc-client/main.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage main\n\nimport (\n\t\"context\"\n\t\"flag\"\n\t\"log\"\n\t\"time\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/google/uuid\"\n\t\"google.golang.org/grpc\"\n\t\"google.golang.org/grpc/credentials/insecure\"\n)\n\nvar remote = flag.String(\"remote\", \"localhost:15002\", \"the remote address of Spark Connect server to connect to\")\n\nfunc main() {\n\topts := []grpc.DialOption{\n\t\tgrpc.WithTransportCredentials(insecure.NewCredentials()),\n\t}\n\n\tconn, err := grpc.NewClient(*remote, opts...)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\tdefer conn.Close()\n\n\tclient := proto.NewSparkConnectServiceClient(conn)\n\n\tctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)\n\tdefer cancel()\n\n\tconfigRequest := proto.ConfigRequest{\n\t\tSessionId: uuid.NewString(),\n\t\tOperation: &proto.ConfigRequest_Operation{\n\t\t\tOpType: &proto.ConfigRequest_Operation_GetAll{\n\t\t\t\tGetAll: &proto.ConfigRequest_GetAll{},\n\t\t\t},\n\t\t},\n\t}\n\tconfigResponse, err := client.Config(ctx, &configRequest)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"configResponse: %v\", configResponse)\n}\n"
  },
  {
    "path": "cmd/spark-connect-example-spark-session/main.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage main\n\nimport (\n\t\"context\"\n\t\"flag\"\n\t\"fmt\"\n\t\"log\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n)\n\nvar (\n\tremote = flag.String(\"remote\", \"sc://localhost:15002\",\n\t\t\"the remote address of Spark Connect server to connect to\")\n\n\tfiledir = flag.String(\"filedir\", \"/tmp\",\n\t\t\"the root directory to save the files generated by this example program\")\n)\n\nfunc main() {\n\tflag.Parse()\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(*remote).Build(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\tdefer utils.WarnOnError(spark.Stop, func(err error) {})\n\n\tdf, err := spark.Sql(ctx, \"select id from range(100)\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, _ = df.FilterByString(ctx, \"id < 10\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Sql(ctx, \"select * from range(100)\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, _ = df.Filter(ctx, functions.Col(\"id\").Lt(functions.Expr(\"10\")))\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, _ = spark.Sql(ctx, \"select * from range(100)\")\n\tdf, err = df.Filter(ctx, functions.Col(\"id\").Lt(functions.Lit(types.Int64(20))))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Sql(ctx, \"select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from sql: select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tschema, err := df.Schema(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tfor _, f := range schema.Fields {\n\t\tlog.Printf(\"Field in dataframe schema: %s - %s\", f.Name, f.DataType.TypeName())\n\t}\n\n\trows, err := df.Collect(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tschema, err = df.Schema(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tfor _, f := range schema.Fields {\n\t\tlog.Printf(\"Field in row: %s - %s\", f.Name, f.DataType.TypeName())\n\t}\n\n\tfor _, row := range rows {\n\t\tlog.Printf(\"Row: %v\", row)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file://%s/spark-connect-write-example-output.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Read().Format(\"parquet\").\n\t\tLoad(fmt.Sprintf(\"file://%s/spark-connect-write-example-output.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from reading parquet\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.CreateTempView(ctx, \"view1\", true, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Sql(ctx, \"select count, word from view1 order by count\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from sql: select count, word from view1 order by count\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"Repartition with one partition\")\n\tdf, err = df.Repartition(ctx, 1, nil)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file://%s/spark-connect-write-example-output-one-partition.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"Repartition with two partitions\")\n\tdf, err = df.Repartition(ctx, 2, nil)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file://%s/spark-connect-write-example-output-two-partition.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"Repartition with columns\")\n\tdf, err = df.Repartition(ctx, 0, []string{\"word\", \"count\"})\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file://%s/spark-connect-write-example-output-repartition-with-column.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"Repartition by range with columns\")\n\tdf, err = df.RepartitionByRange(ctx, 0, functions.Col(\"word\").Desc())\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file:///%s/spark-connect-write-example-output-repartition-by-range-with-column.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n}\n"
  },
  {
    "path": "dev/.rat-excludes",
    "content": ".gitignore\n.gitmodules\n.gitattributes\n.project\ncoverage*\nLICENSE\nNOTICE\nTAGS\nRELEASE\n.*md\n.rat-excludes\nsparksrc\ntarget\ngenerated.out\ngo.sum\ndeps\ncov.report\nbuild.properties\nspark-connect-go.code-workspace"
  },
  {
    "path": "dev/README.md",
    "content": "# Release Script for Apache Spark Connect Go\n\nThis directory contains the release automation script for the Apache Spark Connect Go project.\n\n## Prerequisites\n\n1. **Python Environment**: Create a virtual environment and install dependencies:\n   ```bash\n   python -m venv venv\n   source venv/bin/activate  # On Windows: venv\\Scripts\\activate\n   pip install -r requirements.txt\n   ```\n\n2. **GitHub Token**: Create a GitHub personal access token with the following permissions:\n   - `repo` (Full control of private repositories)\n   - `write:packages` (Upload packages to GitHub Package Registry)\n\n3. **GPG Key**: Ensure you have a GPG key set up for signing:\n   ```bash\n   # List available keys\n   gpg --list-secret-keys\n   \n   # If you don't have a key, create one\n   gpg --gen-key\n   ```\n\n## Usage\n\n```bash\n./release.py --tag <new_tag> --prev-tag <previous_tag> --commit <commit_sha> --gpg-user <gpg_user_id> [options]\n```\n\n### Required Arguments\n\n- `--tag`: The new tag version (e.g., `v0.2.0`)\n- `--prev-tag`: The previous tag version for generating release notes (e.g., `v0.1.0`)\n- `--commit`: The commit SHA that the tag should point to\n- `--gpg-user`: Your GPG user ID for signing (email or key ID)\n\n### Optional Arguments\n\n- `--prerelease`: Mark the release as a pre-release\n- `--repo`: GitHub repository in format `owner/name` (default: `apache/spark-connect-go`)\n- `--token`: GitHub token (alternatively set `GITHUB_TOKEN` environment variable)\n\n### Environment Variables\n\n- `GITHUB_TOKEN`: GitHub personal access token\n\n## Example Usage\n\n```bash\n# Set GitHub token\nexport GITHUB_TOKEN=ghp_your_token_here\n\n# Create a regular release\n./release.py \\\n  --tag v0.2.0 \\\n  --prev-tag v0.1.0 \\\n  --commit abc123def456 \\\n  --gpg-user your.email@example.com\n\n# Create a pre-release\n./release.py \\\n  --tag v0.2.0-rc1 \\\n  --prev-tag v0.1.0 \\\n  --commit abc123def456 \\\n  --gpg-user your.email@example.com \\\n  --prerelease\n```\n\n## What the Script Does\n\n1. **Creates and pushes tag**: Creates a Git tag at the specified commit and pushes it to GitHub\n2. **Generates release notes**: Automatically creates initial release notes from commits between tags\n3. **Interactive input**: Prompts you to enter/modify the release description\n4. **Creates GitHub release**: Creates a draft release on GitHub\n5. **Downloads artifacts**: Downloads the automatically generated source archives (.tar.gz, .zip)\n6. **Signs artifacts**: Creates detached GPG signatures for each artifact\n7. **Verifies signatures**: Confirms that all signatures are valid\n8. **Uploads signatures**: Uploads the signature files to the GitHub release\n\n## Output\n\nThe script creates:\n- A new Git tag pushed to GitHub\n- A draft GitHub release with:\n  - Source code archives (automatically generated by GitHub)\n  - Detached GPG signatures (.asc files)\n  - Release notes based on commits\n\n## Security Notes\n\n- All artifacts are signed with your GPG key\n- Signatures are verified before upload\n- The release is created as a draft first for review\n- Your GPG key must be available in your keyring\n\n## Troubleshooting\n\n### GPG Issues\n```bash\n# If GPG signing fails, check your key\ngpg --list-secret-keys\n\n# Test signing\necho \"test\" | gpg --clearsign --local-user your.email@example.com\n```\n\n### GitHub API Issues\n- Ensure your token has the correct permissions\n- Check rate limits if requests fail\n- Verify repository access\n\n### Git Issues\n- Ensure you're in the correct repository directory\n- Check that the commit SHA exists\n- Verify you have push permissions to the repository"
  },
  {
    "path": "dev/check-license",
    "content": "#!/usr/bin/env bash\n\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\n\n\nacquire_rat_jar () {\n\n  URL=\"${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar\"\n\n  JAR=\"$rat_jar\"\n\n  # Download rat launch jar if it hasn't been downloaded yet\n  if [ ! -f \"$JAR\" ]; then\n    # Download\n    printf \"Attempting to fetch rat\\n\"\n    JAR_DL=\"${JAR}.part\"\n    if [ $(command -v curl) ]; then\n      curl -L --silent \"${URL}\" > \"$JAR_DL\" && mv \"$JAR_DL\" \"$JAR\"\n    elif [ $(command -v wget) ]; then\n      wget --quiet ${URL} -O \"$JAR_DL\" && mv \"$JAR_DL\" \"$JAR\"\n    else\n      printf \"You do not have curl or wget installed, please install rat manually.\\n\"\n      exit -1\n    fi\n  fi\n\n  unzip -tq \"$JAR\" &> /dev/null\n  if [ $? -ne 0 ]; then \n    # We failed to download\n    rm \"$JAR\"\n    printf \"Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\\n\"\n    exit -1\n  fi\n}\n\n# Go to the Spark project root directory\nFWDIR=\"$(cd \"`dirname \"$0\"`\"/..; pwd)\"\ncd \"$FWDIR\"\n\nif test -x \"$JAVA_HOME/bin/java\"; then\n    declare java_cmd=\"$JAVA_HOME/bin/java\"\nelse\n    declare java_cmd=java\nfi\n\nexport RAT_VERSION=0.15\nexport rat_jar=\"$FWDIR\"/lib/apache-rat-${RAT_VERSION}.jar\nmkdir -p \"$FWDIR\"/lib\n\n[[ -f \"$rat_jar\" ]] || acquire_rat_jar || {\n    echo \"Download failed. Obtain the rat jar manually and place it at $rat_jar\"\n    exit 1\n}\n\nmkdir -p target\n$java_cmd -jar \"$rat_jar\" -E \"$FWDIR\"/dev/.rat-excludes -d \"$FWDIR\" > target/rat-results.txt\n\nif [ $? -ne 0 ]; then\n   echo \"RAT exited abnormally\"\n   exit 1\nfi\n\nERRORS=\"$(cat target/rat-results.txt | grep -e \"??\")\"\n\nif test ! -z \"$ERRORS\"; then \n    echo \"Could not find Apache license headers in the following files:\"\n    echo \"$ERRORS\"\n    exit 1\nelse \n    echo -e \"RAT checks passed.\"\nfi\n"
  },
  {
    "path": "dev/gen.py",
    "content": "# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#\thttp://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n\n\n# This is a basic script to generate the builtin functions based on the\n# currently available PySpark installation.\n# Simply call the script as follows:\n#\n# python gen.py > spark/client/functions/generated.go\n\nimport pyspark.sql.connect.functions as F\nimport inspect\nimport typing\nimport types\n\ndef normalize(input: str) -> str:\n    vals = [x[0].upper() + x[1:] for x in input.split(\"_\")]\n    return \"\".join(vals)\n\n\nprint(\"\"\"\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage functions\n\nimport \"github.com/apache/spark-connect-go/spark/sql/column\"\n\"\"\")\n\n\nfor fun in F.__dict__:\n    if fun.startswith(\"_\"):\n        continue\n\n    if not callable(F.__dict__[fun]):\n        continue\n\n    if \"pyspark.sql.connect.functions\" not in F.__dict__[fun].__module__:\n        continue\n\n    if fun == \"expr\" or fun == \"col\" or fun == \"column\" or fun == \"lit\":\n        continue\n\n    # Ignore the aliases of the old distinct.\n    if \"Distinct\" in fun:\n        continue\n\n    sig = inspect.signature(F.__dict__[fun])\n\n    # Ignore all functions that take callables as parameters\n    has_callable = False\n    for p in sig.parameters:\n        param = sig.parameters[p]\n        if \"Callable\" in str(param.annotation):\n            has_callable = True\n            break\n\n    if has_callable:\n        print(f\"// TODO: {fun}: {sig}\")\n        print()\n        continue\n\n    if \"udf\" in fun.lower():\n        print(f\"// Ignore UDF: {fun}: {sig}\")\n        print()\n        continue\n\n    if \"udt\" in fun.lower():\n        print(f\"// Ignore UDT: {fun}: {sig}\")\n        print()\n        continue\n\n    # Convert parameters into Golang\n    res_params = []\n    conversions = []\n    args = []\n    valid = True\n    for p in sig.parameters:\n        param = sig.parameters[p]\n        if param.annotation == inspect.Parameter.empty:\n            res_params.append(f\"{p} interface{{}}\")\n            args.append(p)\n        elif param.kind == inspect.Parameter.VAR_POSITIONAL and param.annotation == \"ColumnOrName\":\n            res_params.append(f\"{p} ...column.Column\")\n            conversions.append(\"vals := make([]column.Column, 0)\")\n            for x in args:\n                conversions.append(f\"vals = append(vals, {x})\")\n            conversions.append(f\"vals = append(vals, {p}...)\")\n            args = [\"vals...\"]\n        elif type(param.annotation) == str and str(param.annotation) == \"ColumnOrName\" and param.kind != inspect.Parameter.VAR_POSITIONAL and param.kind != inspect.Parameter.VAR_KEYWORD:\n            res_params.append(f\"{p} column.Column\")\n            args.append(p)\n        elif len(typing.get_args(param.annotation)) > 1 and typing.ForwardRef(\"ColumnOrName\") in typing.get_args(param.annotation):\n            # Find the parameter with ColumnOrName\n            tmp = [x for x in typing.get_args(param.annotation) if typing.ForwardRef(\"ColumnOrName\") == x]\n            assert len(tmp) == 1\n            res_params.append(f\"{p} column.Column\")\n            args.append(p)\n        elif param.annotation == str or typing.get_args(param.annotation) == (str, types.NoneType):\n            res_params.append(f\"{p} string\")\n            conversions.append(f\"lit_{p} := StringLit({p})\")\n            args.append(f\"lit_{p}\")\n        elif param.annotation == int or typing.get_args(param.annotation) == (int, types.NoneType):\n            res_params.append(f\"{p} int64\")\n            conversions.append(f\"lit_{p} := Int64Lit({p})\")\n            args.append(f\"lit_{p}\")\n        elif param.annotation == float or typing.get_args(param.annotation) == (float, types.NoneType):\n            res_params.append(f\"{p} float64\")\n            conversions.append(f\"lit_{p} := Float64Lit({p})\")\n            args.append(f\"lit_{p}\")\n        else:\n            valid = False\n            break\n\n    if not valid:\n        print(f\"// TODO: {fun}: {sig}\")\n        print()\n    else:\n        name = normalize(fun)\n        # Generate the doc string\n        if F.__dict__[fun].__doc__ is not None:\n            lines = list(map(str.lstrip, F.__dict__[fun].__doc__.split(\"\\n\")))\n            pos = list(map(lambda x: x.startswith(\"..\") or x.startswith(\"Parameters\"), lines)).index(True)\n\n            lines = \"\\n\".join(lines[:pos]).strip().split(\"\\n\")\n            lines[0] = name + \" - \" + lines[0]\n            lines = [\"// \" + l for l in lines]\n            doc = \"\\n\".join(lines) + \"\\n//\"\n            print(doc)\n        print(f\"// {name} is the Golang equivalent of {fun}: {sig}\")\n        print(f\"func {name}({', '.join(res_params)}) column.Column {{\")\n        for c in conversions:\n            print(f\"    {c}\")\n        print(f\"    return column.NewColumn(column.NewUnresolvedFunctionWithColumns(\\\"{fun}\\\", {', '.join(args)}))\")\n        print(f\"}}\")\n        print()"
  },
  {
    "path": "dev/release.py",
    "content": "#!/usr/bin/env python3\n\"\"\"\nLicensed to the Apache Software Foundation (ASF) under one or more\ncontributor license agreements.  See the NOTICE file distributed with\nthis work for additional information regarding copyright ownership.\nThe ASF licenses this file to You under the Apache License, Version 2.0\n(the \"License\"); you may not use this file except in compliance with\nthe License.  You may obtain a copy of the License at\n\n   http://www.apache.org/licenses/LICENSE-2.0\n\nUnless required by applicable law or agreed to in writing, software\ndistributed under the License is distributed on an \"AS IS\" BASIS,\nWITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\nSee the License for the specific language governing permissions and\nlimitations under the License.\n\"\"\"\n\nimport argparse\nimport os\nimport subprocess\nimport sys\nimport tempfile\nimport requests\nfrom pathlib import Path\nfrom typing import List, Dict, Any\n\nimport git\nfrom github import Github\n\n\ndef run_command(cmd: List[str], cwd: str = None, check: bool = True) -> subprocess.CompletedProcess:\n    \"\"\"Run a shell command and return the result.\"\"\"\n    print(f\"Running: {' '.join(cmd)}\")\n    result = subprocess.run(cmd, cwd=cwd, capture_output=True, text=True, check=False)\n\n    if result.returncode != 0 and check:\n        print(f\"Command failed with return code {result.returncode}\")\n        print(f\"STDOUT: {result.stdout}\")\n        print(f\"STDERR: {result.stderr}\")\n        sys.exit(1)\n\n    return result\n\n\ndef get_commits_between_tags(repo_path: str, previous_tag: str, commit_sha: str) -> List[Dict[str, str]]:\n    \"\"\"Get commits between previous tag and current commit.\"\"\"\n    try:\n        repo = git.Repo(repo_path)\n\n        # Get commits from previous tag to current commit\n        commits = list(repo.iter_commits(f\"{previous_tag}..{commit_sha}\"))\n\n        commit_info = []\n        for commit in commits:\n            commit_info.append({\n                'sha': commit.hexsha[:8],  # Short commit ID\n                'author': commit.author.name,\n                'message': commit.message.split('\\n')[0]  # Subject line only\n            })\n\n        return commit_info\n\n    except Exception as e:\n        print(f\"Error getting commits: {e}\")\n        return []\n\n\ndef create_release_notes(commits: List[Dict[str, str]]) -> str:\n    \"\"\"Create initial release notes from commits.\"\"\"\n    if not commits:\n        return \"## Changes\\n\\nNo commits found between releases.\\n\"\n\n    notes = \"## Changes\\n\\n\"\n    for commit in commits:\n        notes += f\"* {commit['sha']} - {commit['message']} ({commit['author']})\\n\"\n\n    return notes\n\n\ndef verify_gpg_key(gpg_user: str) -> bool:\n    \"\"\"Verify that the GPG key exists and can be used for signing.\"\"\"\n    try:\n        result = run_command(['gpg', '--list-secret-keys', gpg_user], check=False)\n        return result.returncode == 0\n    except Exception:\n        return False\n\n\ndef sign_file(file_path: str, gpg_user: str) -> str:\n    \"\"\"Create a detached GPG signature for a file.\"\"\"\n    signature_path = f\"{file_path}.asc\"\n\n    cmd = [\n        'gpg',\n        '--local-user', gpg_user,\n        '--armor',\n        '--detach-sign',\n        file_path\n    ]\n\n    run_command(cmd)\n\n    if not os.path.exists(signature_path):\n        raise RuntimeError(f\"Signature file {signature_path} was not created\")\n\n    return signature_path\n\n\ndef verify_signature(file_path: str, signature_path: str) -> bool:\n    \"\"\"Verify a GPG signature.\"\"\"\n    try:\n        result = run_command(['gpg', '--verify', signature_path, file_path], check=False)\n        return result.returncode == 0\n    except Exception:\n        return False\n\n\ndef download_file(url: str, local_path: str):\n    \"\"\"Download a file from URL to local path.\"\"\"\n    print(f\"Downloading {url} to {local_path}\")\n\n    response = requests.get(url, stream=True)\n    response.raise_for_status()\n\n    with open(local_path, 'wb') as f:\n        for chunk in response.iter_content(chunk_size=8192):\n            f.write(chunk)\n\n\ndef upload_release_asset(release, file_path: str):\n    \"\"\"Upload a file as a release asset.\"\"\"\n    print(f\"Uploading {file_path} to release\")\n\n    filename = os.path.basename(file_path)\n\n    # Use the release object's upload_asset method\n    # PyGithub expects: upload_asset(path, label=None, content_type=None, name=None)\n    release.upload_asset(file_path, label=filename, name=filename)\n\n\ndef main():\n    parser = argparse.ArgumentParser(description='Create and sign Apache Spark Connect Go release')\n    parser.add_argument('--tag', required=True, help='New tag version (e.g., v0.2.0)')\n    parser.add_argument('--prev-tag', required=True, help='Previous tag version (e.g., v0.1.0)')\n    parser.add_argument('--commit', required=True, help='Commit SHA for the tag')\n    parser.add_argument('--gpg-user', required=True, help='GPG user ID for signing')\n    parser.add_argument('--prerelease', action='store_true', help='Mark as pre-release')\n    parser.add_argument('--repo', default='apache/spark-connect-go', help='GitHub repository (owner/name)')\n    parser.add_argument('--token', help='GitHub token (or set GITHUB_TOKEN env var)')\n\n    args = parser.parse_args()\n\n    # Get GitHub token\n    github_token = args.token or os.environ.get('GITHUB_TOKEN')\n    if not github_token:\n        print(\"Error: GitHub token is required. Use --token or set GITHUB_TOKEN environment variable.\")\n        sys.exit(1)\n\n    # Verify GPG key exists\n    if not verify_gpg_key(args.gpg_user):\n        print(f\"Error: GPG key for user '{args.gpg_user}' not found or not usable\")\n        sys.exit(1)\n\n    # Initialize GitHub client\n    github_client = Github(github_token)\n    repo = github_client.get_repo(args.repo)\n\n    print(f\"Creating release for {args.repo}\")\n    print(f\"Tag: {args.tag}\")\n    print(f\"Commit: {args.commit}\")\n    print(f\"Previous tag: {args.prev_tag}\")\n    print(f\"GPG user: {args.gpg_user}\")\n    print(f\"Pre-release: {args.prerelease}\")\n\n    # Step 1: Create and push tag\n    print(\"\\n=== Step 1: Creating and pushing tag ===\")\n    repo_path = os.getcwd()\n\n    try:\n        local_repo = git.Repo(repo_path)\n\n        # Create tag\n        new_tag = local_repo.create_tag(args.tag, ref=args.commit, message=f\"Release {args.tag}\")\n        print(f\"Created tag {args.tag} at commit {args.commit}\")\n\n        # Push tag\n        origin = local_repo.remote('origin')\n        origin.push(new_tag)\n        print(f\"Pushed tag {args.tag} to GitHub\")\n\n    except Exception as e:\n        print(f\"Error creating/pushing tag: {e}\")\n        sys.exit(1)\n\n    # Step 2: Get commits for release notes\n    print(\"\\n=== Step 2: Generating release notes ===\")\n    commits = get_commits_between_tags(repo_path, args.prev_tag, args.commit)\n    initial_release_notes = create_release_notes(commits)\n\n    # Step 3: Prompt user for release description\n    print(\"\\n=== Step 3: Release description ===\")\n    print(\"Initial release notes based on commits:\")\n    print(initial_release_notes)\n    print(\"\\nPlease enter the final release description (press Ctrl+D when done):\")\n\n    lines = []\n    try:\n        while True:\n            line = input()\n            lines.append(line)\n    except EOFError:\n        pass\n\n    # Join the lines and add the initial release notes\n    final_release_notes = '\\n'.join(lines).strip()\n    spacer = \"\\n\\n\" if final_release_notes else \"\"\n    final_release_notes += spacer + initial_release_notes\n\n    # Step 4: Create GitHub release\n    print(\"\\n=== Step 4: Creating GitHub release ===\")\n    try:\n        release = repo.create_git_release(\n            tag=args.tag,\n            name=f\"Release {args.tag}\",\n            message=final_release_notes,\n            draft=True,\n            prerelease=args.prerelease\n        )\n        print(f\"Created draft release: {release.html_url}\")\n\n    except Exception as e:\n        print(f\"Error creating release: {e}\")\n        sys.exit(1)\n\n    # Step 5: Download release artifacts\n    print(\"\\n=== Step 5: Downloading release artifacts ===\")\n\n    # GitHub automatically creates source archives\n    artifacts = [\n        f\"{args.tag}.tar.gz\",\n        f\"{args.tag}.zip\"\n    ]\n\n    with tempfile.TemporaryDirectory() as temp_dir:\n        downloaded_files = []\n\n        for artifact in artifacts:\n            # Construct download URL for source archive\n            download_url = f\"https://github.com/{args.repo}/archive/refs/tags/{artifact}\"\n            local_file = os.path.join(temp_dir, f\"spark-connect-go-{artifact}\")\n\n            try:\n                download_file(download_url, local_file)\n                downloaded_files.append(local_file)\n            except Exception as e:\n                print(f\"Error downloading {artifact}: {e}\")\n                continue\n\n        if not downloaded_files:\n            print(\"Error: No artifacts were downloaded\")\n            sys.exit(1)\n\n        # Step 6: Sign artifacts\n        print(\"\\n=== Step 6: Signing artifacts ===\")\n        signatures = []\n\n        for file_path in downloaded_files:\n            try:\n                print(f\"Signing {os.path.basename(file_path)}\")\n                signature_path = sign_file(file_path, args.gpg_user)\n                signatures.append(signature_path)\n                print(f\"Created signature: {os.path.basename(signature_path)}\")\n\n            except Exception as e:\n                print(f\"Error signing {file_path}: {e}\")\n                continue\n\n        # Step 7: Verify signatures\n        print(\"\\n=== Step 7: Verifying signatures ===\")\n        for i, file_path in enumerate(downloaded_files):\n            if i < len(signatures):\n                signature_path = signatures[i]\n                if verify_signature(file_path, signature_path):\n                    print(f\"✓ Signature verified for {os.path.basename(file_path)}\")\n                else:\n                    print(f\"✗ Signature verification failed for {os.path.basename(file_path)}\")\n                    sys.exit(1)\n\n        # Step 8: Upload signatures to release\n        print(\"\\n=== Step 8: Uploading signatures to release ===\")\n        for signature_path in signatures:\n            try:\n                upload_release_asset(release, signature_path)\n                print(f\"Uploaded {os.path.basename(signature_path)}\")\n            except Exception as e:\n                print(f\"Error uploading {signature_path}: {e}\")\n                continue\n\n    print(f\"\\n=== Release created successfully ===\")\n    print(f\"Release URL: {release.html_url}\")\n    print(f\"Tag: {args.tag}\")\n    print(f\"Status: Draft\")\n    print(f\"Pre-release: {args.prerelease}\")\n    print(\"\\nNext steps:\")\n    print(\"1. Review the release on GitHub\")\n    print(\"2. Test the release artifacts\")\n    print(\"3. Publish the release when ready\")\n\n\nif __name__ == '__main__':\n    main()"
  },
  {
    "path": "dev/requirements.txt",
    "content": "requests>=2.28.0\nPyGithub>=1.58.0\ngitpython>=3.1.30"
  },
  {
    "path": "go.mod",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\nmodule github.com/apache/spark-connect-go\n\ngo 1.23.2\n\nrequire (\n\tgithub.com/apache/arrow-go/v18 v18.4.0\n\tgithub.com/go-errors/errors v1.5.1\n\tgithub.com/google/uuid v1.6.0\n\tgithub.com/stretchr/testify v1.10.0\n\tgoogle.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7\n\tgoogle.golang.org/grpc v1.75.0\n\tgoogle.golang.org/protobuf v1.36.7\n)\n\nrequire (\n\tcloud.google.com/go/compute/metadata v0.7.0 // indirect\n\tgithub.com/kr/pretty v0.3.0 // indirect\n\tgolang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0 // indirect\n\tgolang.org/x/net v0.41.0 // indirect\n\tgolang.org/x/sync v0.15.0 // indirect\n\tgopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect\n)\n\nrequire (\n\tgithub.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect\n\tgithub.com/goccy/go-json v0.10.5 // indirect\n\tgithub.com/google/flatbuffers v25.2.10+incompatible // indirect\n\tgithub.com/klauspost/compress v1.18.0 // indirect\n\tgithub.com/klauspost/cpuid/v2 v2.2.11 // indirect\n\tgithub.com/pierrec/lz4/v4 v4.1.22 // indirect\n\tgithub.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect\n\tgithub.com/zeebo/xxh3 v1.0.2 // indirect\n\tgolang.org/x/mod v0.25.0 // indirect\n\tgolang.org/x/oauth2 v0.30.0\n\tgolang.org/x/sys v0.33.0 // indirect\n\tgolang.org/x/text v0.26.0 // indirect\n\tgolang.org/x/tools v0.34.0 // indirect\n\tgolang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect\n\tgopkg.in/yaml.v3 v3.0.1 // indirect\n)\n"
  },
  {
    "path": "go.sum",
    "content": "cloud.google.com/go/compute/metadata v0.7.0 h1:PBWF+iiAerVNe8UCHxdOt6eHLVc3ydFeOCw78U8ytSU=\ncloud.google.com/go/compute/metadata v0.7.0/go.mod h1:j5MvL9PprKL39t166CoB1uVHfQMs4tFQZZcKwksXUjo=\ngithub.com/andybalholm/brotli v1.2.0 h1:ukwgCxwYrmACq68yiUqwIWnGY0cTPox/M94sVwToPjQ=\ngithub.com/andybalholm/brotli v1.2.0/go.mod h1:rzTDkvFWvIrjDXZHkuS16NPggd91W3kUSvPlQ1pLaKY=\ngithub.com/apache/arrow-go/v18 v18.4.0 h1:/RvkGqH517iY8bZKc4FD5/kkdwXJGjxf28JIXbJ/oB0=\ngithub.com/apache/arrow-go/v18 v18.4.0/go.mod h1:Aawvwhj8x2jURIzD9Moy72cF0FyJXOpkYpdmGRHcw14=\ngithub.com/apache/thrift v0.22.0 h1:r7mTJdj51TMDe6RtcmNdQxgn9XcyfGDOzegMDRg47uc=\ngithub.com/apache/thrift v0.22.0/go.mod h1:1e7J/O1Ae6ZQMTYdy9xa3w9k+XHWPfRvdPyJeynQ+/g=\ngithub.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=\ngithub.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM=\ngithub.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=\ngithub.com/go-errors/errors v1.5.1 h1:ZwEMSLRCapFLflTpT7NKaAc7ukJ8ZPEjzlxt8rPN8bk=\ngithub.com/go-errors/errors v1.5.1/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og=\ngithub.com/go-logr/logr v1.4.3 h1:CjnDlHq8ikf6E492q6eKboGOC0T8CDaOvkHCIg8idEI=\ngithub.com/go-logr/logr v1.4.3/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=\ngithub.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=\ngithub.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=\ngithub.com/goccy/go-json v0.10.5 h1:Fq85nIqj+gXn/S5ahsiTlK3TmC85qgirsdTP/+DeaC4=\ngithub.com/goccy/go-json v0.10.5/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M=\ngithub.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=\ngithub.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=\ngithub.com/golang/snappy v1.0.0 h1:Oy607GVXHs7RtbggtPBnr2RmDArIsAefDwvrdWvRhGs=\ngithub.com/golang/snappy v1.0.0/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=\ngithub.com/google/flatbuffers v25.2.10+incompatible h1:F3vclr7C3HpB1k9mxCGRMXq6FdUalZ6H/pNX4FP1v0Q=\ngithub.com/google/flatbuffers v25.2.10+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8=\ngithub.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8=\ngithub.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU=\ngithub.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=\ngithub.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=\ngithub.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK4=\ngithub.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE=\ngithub.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo=\ngithub.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ=\ngithub.com/klauspost/cpuid/v2 v2.2.11 h1:0OwqZRYI2rFrjS4kvkDnqJkKHdHaRnCm68/DY4OxRzU=\ngithub.com/klauspost/cpuid/v2 v2.2.11/go.mod h1:hqwkgyIinND0mEev00jJYCxPNVRVXFQeu1XKlok6oO0=\ngithub.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=\ngithub.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=\ngithub.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0=\ngithub.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk=\ngithub.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=\ngithub.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=\ngithub.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=\ngithub.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=\ngithub.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=\ngithub.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY=\ngithub.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI=\ngithub.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE=\ngithub.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU=\ngithub.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4=\ngithub.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U=\ngithub.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=\ngithub.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k=\ngithub.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=\ngithub.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA=\ngithub.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY=\ngithub.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ=\ngithub.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0=\ngithub.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0=\ngithub.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA=\ngo.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA=\ngo.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A=\ngo.opentelemetry.io/otel v1.37.0 h1:9zhNfelUvx0KBfu/gb+ZgeAfAgtWrfHJZcAqFC228wQ=\ngo.opentelemetry.io/otel v1.37.0/go.mod h1:ehE/umFRLnuLa/vSccNq9oS1ErUlkkK71gMcN34UG8I=\ngo.opentelemetry.io/otel/metric v1.37.0 h1:mvwbQS5m0tbmqML4NqK+e3aDiO02vsf/WgbsdpcPoZE=\ngo.opentelemetry.io/otel/metric v1.37.0/go.mod h1:04wGrZurHYKOc+RKeye86GwKiTb9FKm1WHtO+4EVr2E=\ngo.opentelemetry.io/otel/sdk v1.37.0 h1:ItB0QUqnjesGRvNcmAcU0LyvkVyGJ2xftD29bWdDvKI=\ngo.opentelemetry.io/otel/sdk v1.37.0/go.mod h1:VredYzxUvuo2q3WRcDnKDjbdvmO0sCzOvVAiY+yUkAg=\ngo.opentelemetry.io/otel/sdk/metric v1.37.0 h1:90lI228XrB9jCMuSdA0673aubgRobVZFhbjxHHspCPc=\ngo.opentelemetry.io/otel/sdk/metric v1.37.0/go.mod h1:cNen4ZWfiD37l5NhS+Keb5RXVWZWpRE+9WyVCpbo5ps=\ngo.opentelemetry.io/otel/trace v1.37.0 h1:HLdcFNbRQBE2imdSEgm/kwqmQj1Or1l/7bW6mxVK7z4=\ngo.opentelemetry.io/otel/trace v1.37.0/go.mod h1:TlgrlQ+PtQO5XFerSPUYG0JSgGyryXewPGyayAWSBS0=\ngolang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0 h1:R84qjqJb5nVJMxqWYb3np9L5ZsaDtB+a39EqjV0JSUM=\ngolang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0/go.mod h1:S9Xr4PYopiDyqSyp5NjCrhFrqg6A5zA2E/iPHPhqnS8=\ngolang.org/x/mod v0.25.0 h1:n7a+ZbQKQA/Ysbyb0/6IbB1H/X41mKgbhfv7AfG/44w=\ngolang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww=\ngolang.org/x/net v0.41.0 h1:vBTly1HeNPEn3wtREYfy4GZ/NECgw2Cnl+nK6Nz3uvw=\ngolang.org/x/net v0.41.0/go.mod h1:B/K4NNqkfmg07DQYrbwvSluqCJOOXwUjeb/5lOisjbA=\ngolang.org/x/oauth2 v0.30.0 h1:dnDm7JmhM45NNpd8FDDeLhK6FwqbOf4MLCM9zb1BOHI=\ngolang.org/x/oauth2 v0.30.0/go.mod h1:B++QgG3ZKulg6sRPGD/mqlHQs5rB3Ml9erfeDY7xKlU=\ngolang.org/x/sync v0.15.0 h1:KWH3jNZsfyT6xfAfKiz6MRNmd46ByHDYaZ7KSkCtdW8=\ngolang.org/x/sync v0.15.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA=\ngolang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw=\ngolang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k=\ngolang.org/x/text v0.26.0 h1:P42AVeLghgTYr4+xUnTRKDMqpar+PtX7KWuNQL21L8M=\ngolang.org/x/text v0.26.0/go.mod h1:QK15LZJUUQVJxhz7wXgxSy/CJaTFjd0G+YLonydOVQA=\ngolang.org/x/tools v0.34.0 h1:qIpSLOxeCYGg9TrcJokLBG4KFA6d795g0xkBkiESGlo=\ngolang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg=\ngolang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY=\ngolang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90=\ngonum.org/v1/gonum v0.16.0 h1:5+ul4Swaf3ESvrOnidPp4GZbzf0mxVQpDCYUQE7OJfk=\ngonum.org/v1/gonum v0.16.0/go.mod h1:fef3am4MQ93R2HHpKnLk4/Tbh/s0+wqD5nfa6Pnwy4E=\ngoogle.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7 h1:pFyd6EwwL2TqFf8emdthzeX+gZE1ElRq3iM8pui4KBY=\ngoogle.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A=\ngoogle.golang.org/grpc v1.75.0 h1:+TW+dqTd2Biwe6KKfhE5JpiYIBWq865PhKGSXiivqt4=\ngoogle.golang.org/grpc v1.75.0/go.mod h1:JtPAzKiq4v1xcAB2hydNlWI2RnF85XXcV0mhKXr2ecQ=\ngoogle.golang.org/protobuf v1.36.7 h1:IgrO7UwFQGJdRNXH/sQux4R1Dj1WAKcLElzeeRaXV2A=\ngoogle.golang.org/protobuf v1.36.7/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY=\ngopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=\ngopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=\ngopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=\ngopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=\ngopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=\ngopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=\ngopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=\n"
  },
  {
    "path": "internal/generated/base.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/base.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\tanypb \"google.golang.org/protobuf/types/known/anypb\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// Plan explanation mode.\ntype AnalyzePlanRequest_Explain_ExplainMode int32\n\nconst (\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED AnalyzePlanRequest_Explain_ExplainMode = 0\n\t// Generates only physical plan.\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_SIMPLE AnalyzePlanRequest_Explain_ExplainMode = 1\n\t// Generates parsed logical plan, analyzed logical plan, optimized logical plan and physical plan.\n\t// Parsed Logical plan is a unresolved plan that extracted from the query. Analyzed logical plans\n\t// transforms which translates unresolvedAttribute and unresolvedRelation into fully typed objects.\n\t// The optimized logical plan transforms through a set of optimization rules, resulting in the\n\t// physical plan.\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_EXTENDED AnalyzePlanRequest_Explain_ExplainMode = 2\n\t// Generates code for the statement, if any and a physical plan.\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_CODEGEN AnalyzePlanRequest_Explain_ExplainMode = 3\n\t// If plan node statistics are available, generates a logical plan and also the statistics.\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_COST AnalyzePlanRequest_Explain_ExplainMode = 4\n\t// Generates a physical plan outline and also node details.\n\tAnalyzePlanRequest_Explain_EXPLAIN_MODE_FORMATTED AnalyzePlanRequest_Explain_ExplainMode = 5\n)\n\n// Enum value maps for AnalyzePlanRequest_Explain_ExplainMode.\nvar (\n\tAnalyzePlanRequest_Explain_ExplainMode_name = map[int32]string{\n\t\t0: \"EXPLAIN_MODE_UNSPECIFIED\",\n\t\t1: \"EXPLAIN_MODE_SIMPLE\",\n\t\t2: \"EXPLAIN_MODE_EXTENDED\",\n\t\t3: \"EXPLAIN_MODE_CODEGEN\",\n\t\t4: \"EXPLAIN_MODE_COST\",\n\t\t5: \"EXPLAIN_MODE_FORMATTED\",\n\t}\n\tAnalyzePlanRequest_Explain_ExplainMode_value = map[string]int32{\n\t\t\"EXPLAIN_MODE_UNSPECIFIED\": 0,\n\t\t\"EXPLAIN_MODE_SIMPLE\":      1,\n\t\t\"EXPLAIN_MODE_EXTENDED\":    2,\n\t\t\"EXPLAIN_MODE_CODEGEN\":     3,\n\t\t\"EXPLAIN_MODE_COST\":        4,\n\t\t\"EXPLAIN_MODE_FORMATTED\":   5,\n\t}\n)\n\nfunc (x AnalyzePlanRequest_Explain_ExplainMode) Enum() *AnalyzePlanRequest_Explain_ExplainMode {\n\tp := new(AnalyzePlanRequest_Explain_ExplainMode)\n\t*p = x\n\treturn p\n}\n\nfunc (x AnalyzePlanRequest_Explain_ExplainMode) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (AnalyzePlanRequest_Explain_ExplainMode) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_base_proto_enumTypes[0].Descriptor()\n}\n\nfunc (AnalyzePlanRequest_Explain_ExplainMode) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_base_proto_enumTypes[0]\n}\n\nfunc (x AnalyzePlanRequest_Explain_ExplainMode) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_Explain_ExplainMode.Descriptor instead.\nfunc (AnalyzePlanRequest_Explain_ExplainMode) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 1, 0}\n}\n\ntype InterruptRequest_InterruptType int32\n\nconst (\n\tInterruptRequest_INTERRUPT_TYPE_UNSPECIFIED InterruptRequest_InterruptType = 0\n\t// Interrupt all running executions within the session with the provided session_id.\n\tInterruptRequest_INTERRUPT_TYPE_ALL InterruptRequest_InterruptType = 1\n\t// Interrupt all running executions within the session with the provided operation_tag.\n\tInterruptRequest_INTERRUPT_TYPE_TAG InterruptRequest_InterruptType = 2\n\t// Interrupt the running execution within the session with the provided operation_id.\n\tInterruptRequest_INTERRUPT_TYPE_OPERATION_ID InterruptRequest_InterruptType = 3\n)\n\n// Enum value maps for InterruptRequest_InterruptType.\nvar (\n\tInterruptRequest_InterruptType_name = map[int32]string{\n\t\t0: \"INTERRUPT_TYPE_UNSPECIFIED\",\n\t\t1: \"INTERRUPT_TYPE_ALL\",\n\t\t2: \"INTERRUPT_TYPE_TAG\",\n\t\t3: \"INTERRUPT_TYPE_OPERATION_ID\",\n\t}\n\tInterruptRequest_InterruptType_value = map[string]int32{\n\t\t\"INTERRUPT_TYPE_UNSPECIFIED\":  0,\n\t\t\"INTERRUPT_TYPE_ALL\":          1,\n\t\t\"INTERRUPT_TYPE_TAG\":          2,\n\t\t\"INTERRUPT_TYPE_OPERATION_ID\": 3,\n\t}\n)\n\nfunc (x InterruptRequest_InterruptType) Enum() *InterruptRequest_InterruptType {\n\tp := new(InterruptRequest_InterruptType)\n\t*p = x\n\treturn p\n}\n\nfunc (x InterruptRequest_InterruptType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (InterruptRequest_InterruptType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_base_proto_enumTypes[1].Descriptor()\n}\n\nfunc (InterruptRequest_InterruptType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_base_proto_enumTypes[1]\n}\n\nfunc (x InterruptRequest_InterruptType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use InterruptRequest_InterruptType.Descriptor instead.\nfunc (InterruptRequest_InterruptType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{13, 0}\n}\n\n// The type of this query context.\ntype FetchErrorDetailsResponse_QueryContext_ContextType int32\n\nconst (\n\tFetchErrorDetailsResponse_QueryContext_SQL       FetchErrorDetailsResponse_QueryContext_ContextType = 0\n\tFetchErrorDetailsResponse_QueryContext_DATAFRAME FetchErrorDetailsResponse_QueryContext_ContextType = 1\n)\n\n// Enum value maps for FetchErrorDetailsResponse_QueryContext_ContextType.\nvar (\n\tFetchErrorDetailsResponse_QueryContext_ContextType_name = map[int32]string{\n\t\t0: \"SQL\",\n\t\t1: \"DATAFRAME\",\n\t}\n\tFetchErrorDetailsResponse_QueryContext_ContextType_value = map[string]int32{\n\t\t\"SQL\":       0,\n\t\t\"DATAFRAME\": 1,\n\t}\n)\n\nfunc (x FetchErrorDetailsResponse_QueryContext_ContextType) Enum() *FetchErrorDetailsResponse_QueryContext_ContextType {\n\tp := new(FetchErrorDetailsResponse_QueryContext_ContextType)\n\t*p = x\n\treturn p\n}\n\nfunc (x FetchErrorDetailsResponse_QueryContext_ContextType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (FetchErrorDetailsResponse_QueryContext_ContextType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_base_proto_enumTypes[2].Descriptor()\n}\n\nfunc (FetchErrorDetailsResponse_QueryContext_ContextType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_base_proto_enumTypes[2]\n}\n\nfunc (x FetchErrorDetailsResponse_QueryContext_ContextType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse_QueryContext_ContextType.Descriptor instead.\nfunc (FetchErrorDetailsResponse_QueryContext_ContextType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22, 1, 0}\n}\n\n// A [[Plan]] is the structure that carries the runtime information for the execution from the\n// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference\n// to the underlying logical plan or it can be of the [[Command]] type that is used to execute\n// commands on the server.\ntype Plan struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to OpType:\n\t//\n\t//\t*Plan_Root\n\t//\t*Plan_Command\n\tOpType isPlan_OpType `protobuf_oneof:\"op_type\"`\n}\n\nfunc (x *Plan) Reset() {\n\t*x = Plan{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Plan) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Plan) ProtoMessage() {}\n\nfunc (x *Plan) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Plan.ProtoReflect.Descriptor instead.\nfunc (*Plan) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *Plan) GetOpType() isPlan_OpType {\n\tif m != nil {\n\t\treturn m.OpType\n\t}\n\treturn nil\n}\n\nfunc (x *Plan) GetRoot() *Relation {\n\tif x, ok := x.GetOpType().(*Plan_Root); ok {\n\t\treturn x.Root\n\t}\n\treturn nil\n}\n\nfunc (x *Plan) GetCommand() *Command {\n\tif x, ok := x.GetOpType().(*Plan_Command); ok {\n\t\treturn x.Command\n\t}\n\treturn nil\n}\n\ntype isPlan_OpType interface {\n\tisPlan_OpType()\n}\n\ntype Plan_Root struct {\n\tRoot *Relation `protobuf:\"bytes,1,opt,name=root,proto3,oneof\"`\n}\n\ntype Plan_Command struct {\n\tCommand *Command `protobuf:\"bytes,2,opt,name=command,proto3,oneof\"`\n}\n\nfunc (*Plan_Root) isPlan_OpType() {}\n\nfunc (*Plan_Command) isPlan_OpType() {}\n\n// User Context is used to refer to one particular user session that is executing\n// queries in the backend.\ntype UserContext struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tUserId   string `protobuf:\"bytes,1,opt,name=user_id,json=userId,proto3\" json:\"user_id,omitempty\"`\n\tUserName string `protobuf:\"bytes,2,opt,name=user_name,json=userName,proto3\" json:\"user_name,omitempty\"`\n\t// To extend the existing user context message that is used to identify incoming requests,\n\t// Spark Connect leverages the Any protobuf type that can be used to inject arbitrary other\n\t// messages into this message. Extensions are stored as a `repeated` type to be able to\n\t// handle multiple active extensions.\n\tExtensions []*anypb.Any `protobuf:\"bytes,999,rep,name=extensions,proto3\" json:\"extensions,omitempty\"`\n}\n\nfunc (x *UserContext) Reset() {\n\t*x = UserContext{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *UserContext) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*UserContext) ProtoMessage() {}\n\nfunc (x *UserContext) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use UserContext.ProtoReflect.Descriptor instead.\nfunc (*UserContext) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *UserContext) GetUserId() string {\n\tif x != nil {\n\t\treturn x.UserId\n\t}\n\treturn \"\"\n}\n\nfunc (x *UserContext) GetUserName() string {\n\tif x != nil {\n\t\treturn x.UserName\n\t}\n\treturn \"\"\n}\n\nfunc (x *UserContext) GetExtensions() []*anypb.Any {\n\tif x != nil {\n\t\treturn x.Extensions\n\t}\n\treturn nil\n}\n\n// Request to perform plan analyze, optionally to explain the plan.\ntype AnalyzePlanRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,17,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,3,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// Types that are assignable to Analyze:\n\t//\n\t//\t*AnalyzePlanRequest_Schema_\n\t//\t*AnalyzePlanRequest_Explain_\n\t//\t*AnalyzePlanRequest_TreeString_\n\t//\t*AnalyzePlanRequest_IsLocal_\n\t//\t*AnalyzePlanRequest_IsStreaming_\n\t//\t*AnalyzePlanRequest_InputFiles_\n\t//\t*AnalyzePlanRequest_SparkVersion_\n\t//\t*AnalyzePlanRequest_DdlParse\n\t//\t*AnalyzePlanRequest_SameSemantics_\n\t//\t*AnalyzePlanRequest_SemanticHash_\n\t//\t*AnalyzePlanRequest_Persist_\n\t//\t*AnalyzePlanRequest_Unpersist_\n\t//\t*AnalyzePlanRequest_GetStorageLevel_\n\t//\t*AnalyzePlanRequest_JsonToDdl\n\tAnalyze isAnalyzePlanRequest_Analyze `protobuf_oneof:\"analyze\"`\n}\n\nfunc (x *AnalyzePlanRequest) Reset() {\n\t*x = AnalyzePlanRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *AnalyzePlanRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AnalyzePlanRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AnalyzePlanRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (m *AnalyzePlanRequest) GetAnalyze() isAnalyzePlanRequest_Analyze {\n\tif m != nil {\n\t\treturn m.Analyze\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetSchema() *AnalyzePlanRequest_Schema {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Schema_); ok {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetExplain() *AnalyzePlanRequest_Explain {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Explain_); ok {\n\t\treturn x.Explain\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetTreeString() *AnalyzePlanRequest_TreeString {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_TreeString_); ok {\n\t\treturn x.TreeString\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetIsLocal() *AnalyzePlanRequest_IsLocal {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_IsLocal_); ok {\n\t\treturn x.IsLocal\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetIsStreaming() *AnalyzePlanRequest_IsStreaming {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_IsStreaming_); ok {\n\t\treturn x.IsStreaming\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetInputFiles() *AnalyzePlanRequest_InputFiles {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_InputFiles_); ok {\n\t\treturn x.InputFiles\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetSparkVersion() *AnalyzePlanRequest_SparkVersion {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SparkVersion_); ok {\n\t\treturn x.SparkVersion\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetDdlParse() *AnalyzePlanRequest_DDLParse {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_DdlParse); ok {\n\t\treturn x.DdlParse\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetSameSemantics() *AnalyzePlanRequest_SameSemantics {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SameSemantics_); ok {\n\t\treturn x.SameSemantics\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetSemanticHash() *AnalyzePlanRequest_SemanticHash {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SemanticHash_); ok {\n\t\treturn x.SemanticHash\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetPersist() *AnalyzePlanRequest_Persist {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Persist_); ok {\n\t\treturn x.Persist\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetUnpersist() *AnalyzePlanRequest_Unpersist {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Unpersist_); ok {\n\t\treturn x.Unpersist\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetGetStorageLevel() *AnalyzePlanRequest_GetStorageLevel {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_GetStorageLevel_); ok {\n\t\treturn x.GetStorageLevel\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest) GetJsonToDdl() *AnalyzePlanRequest_JsonToDDL {\n\tif x, ok := x.GetAnalyze().(*AnalyzePlanRequest_JsonToDdl); ok {\n\t\treturn x.JsonToDdl\n\t}\n\treturn nil\n}\n\ntype isAnalyzePlanRequest_Analyze interface {\n\tisAnalyzePlanRequest_Analyze()\n}\n\ntype AnalyzePlanRequest_Schema_ struct {\n\tSchema *AnalyzePlanRequest_Schema `protobuf:\"bytes,4,opt,name=schema,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_Explain_ struct {\n\tExplain *AnalyzePlanRequest_Explain `protobuf:\"bytes,5,opt,name=explain,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_TreeString_ struct {\n\tTreeString *AnalyzePlanRequest_TreeString `protobuf:\"bytes,6,opt,name=tree_string,json=treeString,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_IsLocal_ struct {\n\tIsLocal *AnalyzePlanRequest_IsLocal `protobuf:\"bytes,7,opt,name=is_local,json=isLocal,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_IsStreaming_ struct {\n\tIsStreaming *AnalyzePlanRequest_IsStreaming `protobuf:\"bytes,8,opt,name=is_streaming,json=isStreaming,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_InputFiles_ struct {\n\tInputFiles *AnalyzePlanRequest_InputFiles `protobuf:\"bytes,9,opt,name=input_files,json=inputFiles,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_SparkVersion_ struct {\n\tSparkVersion *AnalyzePlanRequest_SparkVersion `protobuf:\"bytes,10,opt,name=spark_version,json=sparkVersion,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_DdlParse struct {\n\tDdlParse *AnalyzePlanRequest_DDLParse `protobuf:\"bytes,11,opt,name=ddl_parse,json=ddlParse,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_SameSemantics_ struct {\n\tSameSemantics *AnalyzePlanRequest_SameSemantics `protobuf:\"bytes,12,opt,name=same_semantics,json=sameSemantics,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_SemanticHash_ struct {\n\tSemanticHash *AnalyzePlanRequest_SemanticHash `protobuf:\"bytes,13,opt,name=semantic_hash,json=semanticHash,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_Persist_ struct {\n\tPersist *AnalyzePlanRequest_Persist `protobuf:\"bytes,14,opt,name=persist,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_Unpersist_ struct {\n\tUnpersist *AnalyzePlanRequest_Unpersist `protobuf:\"bytes,15,opt,name=unpersist,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_GetStorageLevel_ struct {\n\tGetStorageLevel *AnalyzePlanRequest_GetStorageLevel `protobuf:\"bytes,16,opt,name=get_storage_level,json=getStorageLevel,proto3,oneof\"`\n}\n\ntype AnalyzePlanRequest_JsonToDdl struct {\n\tJsonToDdl *AnalyzePlanRequest_JsonToDDL `protobuf:\"bytes,18,opt,name=json_to_ddl,json=jsonToDdl,proto3,oneof\"`\n}\n\nfunc (*AnalyzePlanRequest_Schema_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_Explain_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_TreeString_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_IsLocal_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_IsStreaming_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_InputFiles_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_SparkVersion_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_DdlParse) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_SameSemantics_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_SemanticHash_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_Persist_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_Unpersist_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_GetStorageLevel_) isAnalyzePlanRequest_Analyze() {}\n\nfunc (*AnalyzePlanRequest_JsonToDdl) isAnalyzePlanRequest_Analyze() {}\n\n// Response to performing analysis of the query. Contains relevant metadata to be able to\n// reason about the performance.\n// Next ID: 16\ntype AnalyzePlanResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,15,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// Types that are assignable to Result:\n\t//\n\t//\t*AnalyzePlanResponse_Schema_\n\t//\t*AnalyzePlanResponse_Explain_\n\t//\t*AnalyzePlanResponse_TreeString_\n\t//\t*AnalyzePlanResponse_IsLocal_\n\t//\t*AnalyzePlanResponse_IsStreaming_\n\t//\t*AnalyzePlanResponse_InputFiles_\n\t//\t*AnalyzePlanResponse_SparkVersion_\n\t//\t*AnalyzePlanResponse_DdlParse\n\t//\t*AnalyzePlanResponse_SameSemantics_\n\t//\t*AnalyzePlanResponse_SemanticHash_\n\t//\t*AnalyzePlanResponse_Persist_\n\t//\t*AnalyzePlanResponse_Unpersist_\n\t//\t*AnalyzePlanResponse_GetStorageLevel_\n\t//\t*AnalyzePlanResponse_JsonToDdl\n\tResult isAnalyzePlanResponse_Result `protobuf_oneof:\"result\"`\n}\n\nfunc (x *AnalyzePlanResponse) Reset() {\n\t*x = AnalyzePlanResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *AnalyzePlanResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AnalyzePlanResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (m *AnalyzePlanResponse) GetResult() isAnalyzePlanResponse_Result {\n\tif m != nil {\n\t\treturn m.Result\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetSchema() *AnalyzePlanResponse_Schema {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_Schema_); ok {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetExplain() *AnalyzePlanResponse_Explain {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_Explain_); ok {\n\t\treturn x.Explain\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetTreeString() *AnalyzePlanResponse_TreeString {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_TreeString_); ok {\n\t\treturn x.TreeString\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetIsLocal() *AnalyzePlanResponse_IsLocal {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_IsLocal_); ok {\n\t\treturn x.IsLocal\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetIsStreaming() *AnalyzePlanResponse_IsStreaming {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_IsStreaming_); ok {\n\t\treturn x.IsStreaming\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetInputFiles() *AnalyzePlanResponse_InputFiles {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_InputFiles_); ok {\n\t\treturn x.InputFiles\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetSparkVersion() *AnalyzePlanResponse_SparkVersion {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_SparkVersion_); ok {\n\t\treturn x.SparkVersion\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetDdlParse() *AnalyzePlanResponse_DDLParse {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_DdlParse); ok {\n\t\treturn x.DdlParse\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetSameSemantics() *AnalyzePlanResponse_SameSemantics {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_SameSemantics_); ok {\n\t\treturn x.SameSemantics\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetSemanticHash() *AnalyzePlanResponse_SemanticHash {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_SemanticHash_); ok {\n\t\treturn x.SemanticHash\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetPersist() *AnalyzePlanResponse_Persist {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_Persist_); ok {\n\t\treturn x.Persist\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetUnpersist() *AnalyzePlanResponse_Unpersist {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_Unpersist_); ok {\n\t\treturn x.Unpersist\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetGetStorageLevel() *AnalyzePlanResponse_GetStorageLevel {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_GetStorageLevel_); ok {\n\t\treturn x.GetStorageLevel\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanResponse) GetJsonToDdl() *AnalyzePlanResponse_JsonToDDL {\n\tif x, ok := x.GetResult().(*AnalyzePlanResponse_JsonToDdl); ok {\n\t\treturn x.JsonToDdl\n\t}\n\treturn nil\n}\n\ntype isAnalyzePlanResponse_Result interface {\n\tisAnalyzePlanResponse_Result()\n}\n\ntype AnalyzePlanResponse_Schema_ struct {\n\tSchema *AnalyzePlanResponse_Schema `protobuf:\"bytes,2,opt,name=schema,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_Explain_ struct {\n\tExplain *AnalyzePlanResponse_Explain `protobuf:\"bytes,3,opt,name=explain,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_TreeString_ struct {\n\tTreeString *AnalyzePlanResponse_TreeString `protobuf:\"bytes,4,opt,name=tree_string,json=treeString,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_IsLocal_ struct {\n\tIsLocal *AnalyzePlanResponse_IsLocal `protobuf:\"bytes,5,opt,name=is_local,json=isLocal,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_IsStreaming_ struct {\n\tIsStreaming *AnalyzePlanResponse_IsStreaming `protobuf:\"bytes,6,opt,name=is_streaming,json=isStreaming,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_InputFiles_ struct {\n\tInputFiles *AnalyzePlanResponse_InputFiles `protobuf:\"bytes,7,opt,name=input_files,json=inputFiles,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_SparkVersion_ struct {\n\tSparkVersion *AnalyzePlanResponse_SparkVersion `protobuf:\"bytes,8,opt,name=spark_version,json=sparkVersion,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_DdlParse struct {\n\tDdlParse *AnalyzePlanResponse_DDLParse `protobuf:\"bytes,9,opt,name=ddl_parse,json=ddlParse,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_SameSemantics_ struct {\n\tSameSemantics *AnalyzePlanResponse_SameSemantics `protobuf:\"bytes,10,opt,name=same_semantics,json=sameSemantics,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_SemanticHash_ struct {\n\tSemanticHash *AnalyzePlanResponse_SemanticHash `protobuf:\"bytes,11,opt,name=semantic_hash,json=semanticHash,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_Persist_ struct {\n\tPersist *AnalyzePlanResponse_Persist `protobuf:\"bytes,12,opt,name=persist,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_Unpersist_ struct {\n\tUnpersist *AnalyzePlanResponse_Unpersist `protobuf:\"bytes,13,opt,name=unpersist,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_GetStorageLevel_ struct {\n\tGetStorageLevel *AnalyzePlanResponse_GetStorageLevel `protobuf:\"bytes,14,opt,name=get_storage_level,json=getStorageLevel,proto3,oneof\"`\n}\n\ntype AnalyzePlanResponse_JsonToDdl struct {\n\tJsonToDdl *AnalyzePlanResponse_JsonToDDL `protobuf:\"bytes,16,opt,name=json_to_ddl,json=jsonToDdl,proto3,oneof\"`\n}\n\nfunc (*AnalyzePlanResponse_Schema_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_Explain_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_TreeString_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_IsLocal_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_IsStreaming_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_InputFiles_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_SparkVersion_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_DdlParse) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_SameSemantics_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_SemanticHash_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_Persist_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_Unpersist_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_GetStorageLevel_) isAnalyzePlanResponse_Result() {}\n\nfunc (*AnalyzePlanResponse_JsonToDdl) isAnalyzePlanResponse_Result() {}\n\n// A request to be executed by the service.\ntype ExecutePlanRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,8,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\t//\n\t// user_context.user_id and session+id both identify a unique remote spark session on the\n\t// server side.\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Optional)\n\t// Provide an id for this request. If not provided, it will be generated by the server.\n\t// It is returned in every ExecutePlanResponse.operation_id of the ExecutePlan response stream.\n\t// The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tOperationId *string `protobuf:\"bytes,6,opt,name=operation_id,json=operationId,proto3,oneof\" json:\"operation_id,omitempty\"`\n\t// (Required) The logical plan to be executed / analyzed.\n\tPlan *Plan `protobuf:\"bytes,3,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,4,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// Repeated element for options that can be passed to the request. This element is currently\n\t// unused but allows to pass in an extension value used for arbitrary options.\n\tRequestOptions []*ExecutePlanRequest_RequestOption `protobuf:\"bytes,5,rep,name=request_options,json=requestOptions,proto3\" json:\"request_options,omitempty\"`\n\t// Tags to tag the given execution with.\n\t// Tags cannot contain ',' character and cannot be empty strings.\n\t// Used by Interrupt with interrupt.tag.\n\tTags []string `protobuf:\"bytes,7,rep,name=tags,proto3\" json:\"tags,omitempty\"`\n}\n\nfunc (x *ExecutePlanRequest) Reset() {\n\t*x = ExecutePlanRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanRequest) ProtoMessage() {}\n\nfunc (x *ExecutePlanRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanRequest.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *ExecutePlanRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanRequest) GetOperationId() string {\n\tif x != nil && x.OperationId != nil {\n\t\treturn *x.OperationId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanRequest) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanRequest) GetRequestOptions() []*ExecutePlanRequest_RequestOption {\n\tif x != nil {\n\t\treturn x.RequestOptions\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanRequest) GetTags() []string {\n\tif x != nil {\n\t\treturn x.Tags\n\t}\n\treturn nil\n}\n\n// The response of a query, can be one or more for each request. Responses belonging to the\n// same input query, carry the same `session_id`.\n// Next ID: 17\ntype ExecutePlanResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,15,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// Identifies the ExecutePlan execution.\n\t// If set by the client in ExecutePlanRequest.operationId, that value is returned.\n\t// Otherwise generated by the server.\n\t// It is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tOperationId string `protobuf:\"bytes,12,opt,name=operation_id,json=operationId,proto3\" json:\"operation_id,omitempty\"`\n\t// Identified the response in the stream.\n\t// The id is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tResponseId string `protobuf:\"bytes,13,opt,name=response_id,json=responseId,proto3\" json:\"response_id,omitempty\"`\n\t// Union type for the different response messages.\n\t//\n\t// Types that are assignable to ResponseType:\n\t//\n\t//\t*ExecutePlanResponse_ArrowBatch_\n\t//\t*ExecutePlanResponse_SqlCommandResult_\n\t//\t*ExecutePlanResponse_WriteStreamOperationStartResult\n\t//\t*ExecutePlanResponse_StreamingQueryCommandResult\n\t//\t*ExecutePlanResponse_GetResourcesCommandResult\n\t//\t*ExecutePlanResponse_StreamingQueryManagerCommandResult\n\t//\t*ExecutePlanResponse_StreamingQueryListenerEventsResult\n\t//\t*ExecutePlanResponse_ResultComplete_\n\t//\t*ExecutePlanResponse_CreateResourceProfileCommandResult\n\t//\t*ExecutePlanResponse_ExecutionProgress_\n\t//\t*ExecutePlanResponse_CheckpointCommandResult\n\t//\t*ExecutePlanResponse_MlCommandResult\n\t//\t*ExecutePlanResponse_PipelineEventResult\n\t//\t*ExecutePlanResponse_PipelineCommandResult\n\t//\t*ExecutePlanResponse_Extension\n\tResponseType isExecutePlanResponse_ResponseType `protobuf_oneof:\"response_type\"`\n\t// Metrics for the query execution. Typically, this field is only present in the last\n\t// batch of results and then represent the overall state of the query execution.\n\tMetrics *ExecutePlanResponse_Metrics `protobuf:\"bytes,4,opt,name=metrics,proto3\" json:\"metrics,omitempty\"`\n\t// The metrics observed during the execution of the query plan.\n\tObservedMetrics []*ExecutePlanResponse_ObservedMetrics `protobuf:\"bytes,6,rep,name=observed_metrics,json=observedMetrics,proto3\" json:\"observed_metrics,omitempty\"`\n\t// (Optional) The Spark schema. This field is available when `collect` is called.\n\tSchema *DataType `protobuf:\"bytes,7,opt,name=schema,proto3\" json:\"schema,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse) Reset() {\n\t*x = ExecutePlanResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (x *ExecutePlanResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse) GetOperationId() string {\n\tif x != nil {\n\t\treturn x.OperationId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse) GetResponseId() string {\n\tif x != nil {\n\t\treturn x.ResponseId\n\t}\n\treturn \"\"\n}\n\nfunc (m *ExecutePlanResponse) GetResponseType() isExecutePlanResponse_ResponseType {\n\tif m != nil {\n\t\treturn m.ResponseType\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetArrowBatch() *ExecutePlanResponse_ArrowBatch {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_ArrowBatch_); ok {\n\t\treturn x.ArrowBatch\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetSqlCommandResult() *ExecutePlanResponse_SqlCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_SqlCommandResult_); ok {\n\t\treturn x.SqlCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetWriteStreamOperationStartResult() *WriteStreamOperationStartResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_WriteStreamOperationStartResult); ok {\n\t\treturn x.WriteStreamOperationStartResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetStreamingQueryCommandResult() *StreamingQueryCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryCommandResult); ok {\n\t\treturn x.StreamingQueryCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetGetResourcesCommandResult() *GetResourcesCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_GetResourcesCommandResult); ok {\n\t\treturn x.GetResourcesCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetStreamingQueryManagerCommandResult() *StreamingQueryManagerCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryManagerCommandResult); ok {\n\t\treturn x.StreamingQueryManagerCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetStreamingQueryListenerEventsResult() *StreamingQueryListenerEventsResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryListenerEventsResult); ok {\n\t\treturn x.StreamingQueryListenerEventsResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetResultComplete() *ExecutePlanResponse_ResultComplete {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_ResultComplete_); ok {\n\t\treturn x.ResultComplete\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetCreateResourceProfileCommandResult() *CreateResourceProfileCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_CreateResourceProfileCommandResult); ok {\n\t\treturn x.CreateResourceProfileCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetExecutionProgress() *ExecutePlanResponse_ExecutionProgress {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_ExecutionProgress_); ok {\n\t\treturn x.ExecutionProgress\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetCheckpointCommandResult() *CheckpointCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_CheckpointCommandResult); ok {\n\t\treturn x.CheckpointCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetMlCommandResult() *MlCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_MlCommandResult); ok {\n\t\treturn x.MlCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetPipelineEventResult() *PipelineEventResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_PipelineEventResult); ok {\n\t\treturn x.PipelineEventResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetPipelineCommandResult() *PipelineCommandResult {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_PipelineCommandResult); ok {\n\t\treturn x.PipelineCommandResult\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetExtension() *anypb.Any {\n\tif x, ok := x.GetResponseType().(*ExecutePlanResponse_Extension); ok {\n\t\treturn x.Extension\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetMetrics() *ExecutePlanResponse_Metrics {\n\tif x != nil {\n\t\treturn x.Metrics\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetObservedMetrics() []*ExecutePlanResponse_ObservedMetrics {\n\tif x != nil {\n\t\treturn x.ObservedMetrics\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\ntype isExecutePlanResponse_ResponseType interface {\n\tisExecutePlanResponse_ResponseType()\n}\n\ntype ExecutePlanResponse_ArrowBatch_ struct {\n\tArrowBatch *ExecutePlanResponse_ArrowBatch `protobuf:\"bytes,2,opt,name=arrow_batch,json=arrowBatch,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_SqlCommandResult_ struct {\n\t// Special case for executing SQL commands.\n\tSqlCommandResult *ExecutePlanResponse_SqlCommandResult `protobuf:\"bytes,5,opt,name=sql_command_result,json=sqlCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_WriteStreamOperationStartResult struct {\n\t// Response for a streaming query.\n\tWriteStreamOperationStartResult *WriteStreamOperationStartResult `protobuf:\"bytes,8,opt,name=write_stream_operation_start_result,json=writeStreamOperationStartResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_StreamingQueryCommandResult struct {\n\t// Response for commands on a streaming query.\n\tStreamingQueryCommandResult *StreamingQueryCommandResult `protobuf:\"bytes,9,opt,name=streaming_query_command_result,json=streamingQueryCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_GetResourcesCommandResult struct {\n\t// Response for 'SparkContext.resources'.\n\tGetResourcesCommandResult *GetResourcesCommandResult `protobuf:\"bytes,10,opt,name=get_resources_command_result,json=getResourcesCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_StreamingQueryManagerCommandResult struct {\n\t// Response for commands on the streaming query manager.\n\tStreamingQueryManagerCommandResult *StreamingQueryManagerCommandResult `protobuf:\"bytes,11,opt,name=streaming_query_manager_command_result,json=streamingQueryManagerCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_StreamingQueryListenerEventsResult struct {\n\t// Response for commands on the client side streaming query listener.\n\tStreamingQueryListenerEventsResult *StreamingQueryListenerEventsResult `protobuf:\"bytes,16,opt,name=streaming_query_listener_events_result,json=streamingQueryListenerEventsResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_ResultComplete_ struct {\n\t// Response type informing if the stream is complete in reattachable execution.\n\tResultComplete *ExecutePlanResponse_ResultComplete `protobuf:\"bytes,14,opt,name=result_complete,json=resultComplete,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_CreateResourceProfileCommandResult struct {\n\t// Response for command that creates ResourceProfile.\n\tCreateResourceProfileCommandResult *CreateResourceProfileCommandResult `protobuf:\"bytes,17,opt,name=create_resource_profile_command_result,json=createResourceProfileCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_ExecutionProgress_ struct {\n\t// (Optional) Intermediate query progress reports.\n\tExecutionProgress *ExecutePlanResponse_ExecutionProgress `protobuf:\"bytes,18,opt,name=execution_progress,json=executionProgress,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_CheckpointCommandResult struct {\n\t// Response for command that checkpoints a DataFrame.\n\tCheckpointCommandResult *CheckpointCommandResult `protobuf:\"bytes,19,opt,name=checkpoint_command_result,json=checkpointCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_MlCommandResult struct {\n\t// ML command response\n\tMlCommandResult *MlCommandResult `protobuf:\"bytes,20,opt,name=ml_command_result,json=mlCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_PipelineEventResult struct {\n\t// Response containing pipeline event that is streamed back to the client during a pipeline run\n\tPipelineEventResult *PipelineEventResult `protobuf:\"bytes,21,opt,name=pipeline_event_result,json=pipelineEventResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_PipelineCommandResult struct {\n\t// Pipeline command response\n\tPipelineCommandResult *PipelineCommandResult `protobuf:\"bytes,22,opt,name=pipeline_command_result,json=pipelineCommandResult,proto3,oneof\"`\n}\n\ntype ExecutePlanResponse_Extension struct {\n\t// Support arbitrary result objects.\n\tExtension *anypb.Any `protobuf:\"bytes,999,opt,name=extension,proto3,oneof\"`\n}\n\nfunc (*ExecutePlanResponse_ArrowBatch_) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_SqlCommandResult_) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_WriteStreamOperationStartResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_StreamingQueryCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_GetResourcesCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_StreamingQueryManagerCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_StreamingQueryListenerEventsResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_ResultComplete_) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_CreateResourceProfileCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_ExecutionProgress_) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_CheckpointCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_MlCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_PipelineEventResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_PipelineCommandResult) isExecutePlanResponse_ResponseType() {}\n\nfunc (*ExecutePlanResponse_Extension) isExecutePlanResponse_ResponseType() {}\n\n// The key-value pair for the config request and response.\ntype KeyValue struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The key.\n\tKey string `protobuf:\"bytes,1,opt,name=key,proto3\" json:\"key,omitempty\"`\n\t// (Optional) The value.\n\tValue *string `protobuf:\"bytes,2,opt,name=value,proto3,oneof\" json:\"value,omitempty\"`\n}\n\nfunc (x *KeyValue) Reset() {\n\t*x = KeyValue{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *KeyValue) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*KeyValue) ProtoMessage() {}\n\nfunc (x *KeyValue) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use KeyValue.ProtoReflect.Descriptor instead.\nfunc (*KeyValue) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (x *KeyValue) GetKey() string {\n\tif x != nil {\n\t\treturn x.Key\n\t}\n\treturn \"\"\n}\n\nfunc (x *KeyValue) GetValue() string {\n\tif x != nil && x.Value != nil {\n\t\treturn *x.Value\n\t}\n\treturn \"\"\n}\n\n// Request to update or fetch the configurations.\ntype ConfigRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,8,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Required) The operation for the config.\n\tOperation *ConfigRequest_Operation `protobuf:\"bytes,3,opt,name=operation,proto3\" json:\"operation,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,4,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n}\n\nfunc (x *ConfigRequest) Reset() {\n\t*x = ConfigRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest) ProtoMessage() {}\n\nfunc (x *ConfigRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (x *ConfigRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ConfigRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ConfigRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest) GetOperation() *ConfigRequest_Operation {\n\tif x != nil {\n\t\treturn x.Operation\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\n// Response to the config request.\n// Next ID: 5\ntype ConfigResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,4,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// (Optional) The result key-value pairs.\n\t//\n\t// Available when the operation is 'Get', 'GetWithDefault', 'GetOption', 'GetAll'.\n\t// Also available for the operation 'IsModifiable' with boolean string \"true\" and \"false\".\n\tPairs []*KeyValue `protobuf:\"bytes,2,rep,name=pairs,proto3\" json:\"pairs,omitempty\"`\n\t// (Optional)\n\t//\n\t// Warning messages for deprecated or unsupported configurations.\n\tWarnings []string `protobuf:\"bytes,3,rep,name=warnings,proto3\" json:\"warnings,omitempty\"`\n}\n\nfunc (x *ConfigResponse) Reset() {\n\t*x = ConfigResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigResponse) ProtoMessage() {}\n\nfunc (x *ConfigResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigResponse.ProtoReflect.Descriptor instead.\nfunc (*ConfigResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *ConfigResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ConfigResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ConfigResponse) GetPairs() []*KeyValue {\n\tif x != nil {\n\t\treturn x.Pairs\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigResponse) GetWarnings() []string {\n\tif x != nil {\n\t\treturn x.Warnings\n\t}\n\treturn nil\n}\n\n// Request to transfer client-local artifacts.\ntype AddArtifactsRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,6,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// The payload is either a batch of artifacts or a partial chunk of a large artifact.\n\t//\n\t// Types that are assignable to Payload:\n\t//\n\t//\t*AddArtifactsRequest_Batch_\n\t//\t*AddArtifactsRequest_BeginChunk\n\t//\t*AddArtifactsRequest_Chunk\n\tPayload isAddArtifactsRequest_Payload `protobuf_oneof:\"payload\"`\n}\n\nfunc (x *AddArtifactsRequest) Reset() {\n\t*x = AddArtifactsRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsRequest) ProtoMessage() {}\n\nfunc (x *AddArtifactsRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsRequest.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *AddArtifactsRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *AddArtifactsRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (m *AddArtifactsRequest) GetPayload() isAddArtifactsRequest_Payload {\n\tif m != nil {\n\t\treturn m.Payload\n\t}\n\treturn nil\n}\n\nfunc (x *AddArtifactsRequest) GetBatch() *AddArtifactsRequest_Batch {\n\tif x, ok := x.GetPayload().(*AddArtifactsRequest_Batch_); ok {\n\t\treturn x.Batch\n\t}\n\treturn nil\n}\n\nfunc (x *AddArtifactsRequest) GetBeginChunk() *AddArtifactsRequest_BeginChunkedArtifact {\n\tif x, ok := x.GetPayload().(*AddArtifactsRequest_BeginChunk); ok {\n\t\treturn x.BeginChunk\n\t}\n\treturn nil\n}\n\nfunc (x *AddArtifactsRequest) GetChunk() *AddArtifactsRequest_ArtifactChunk {\n\tif x, ok := x.GetPayload().(*AddArtifactsRequest_Chunk); ok {\n\t\treturn x.Chunk\n\t}\n\treturn nil\n}\n\ntype isAddArtifactsRequest_Payload interface {\n\tisAddArtifactsRequest_Payload()\n}\n\ntype AddArtifactsRequest_Batch_ struct {\n\tBatch *AddArtifactsRequest_Batch `protobuf:\"bytes,3,opt,name=batch,proto3,oneof\"`\n}\n\ntype AddArtifactsRequest_BeginChunk struct {\n\t// The metadata and the initial chunk of a large artifact chunked into multiple requests.\n\t// The server side is notified about the total size of the large artifact as well as the\n\t// number of chunks to expect.\n\tBeginChunk *AddArtifactsRequest_BeginChunkedArtifact `protobuf:\"bytes,4,opt,name=begin_chunk,json=beginChunk,proto3,oneof\"`\n}\n\ntype AddArtifactsRequest_Chunk struct {\n\t// A chunk of an artifact excluding metadata. This can be any chunk of a large artifact\n\t// excluding the first chunk (which is included in `BeginChunkedArtifact`).\n\tChunk *AddArtifactsRequest_ArtifactChunk `protobuf:\"bytes,5,opt,name=chunk,proto3,oneof\"`\n}\n\nfunc (*AddArtifactsRequest_Batch_) isAddArtifactsRequest_Payload() {}\n\nfunc (*AddArtifactsRequest_BeginChunk) isAddArtifactsRequest_Payload() {}\n\nfunc (*AddArtifactsRequest_Chunk) isAddArtifactsRequest_Payload() {}\n\n// Response to adding an artifact. Contains relevant metadata to verify successful transfer of\n// artifact(s).\n// Next ID: 4\ntype AddArtifactsResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Session id in which the AddArtifact was running.\n\tSessionId string `protobuf:\"bytes,2,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// The list of artifact(s) seen by the server.\n\tArtifacts []*AddArtifactsResponse_ArtifactSummary `protobuf:\"bytes,1,rep,name=artifacts,proto3\" json:\"artifacts,omitempty\"`\n}\n\nfunc (x *AddArtifactsResponse) Reset() {\n\t*x = AddArtifactsResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsResponse) ProtoMessage() {}\n\nfunc (x *AddArtifactsResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsResponse.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *AddArtifactsResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsResponse) GetArtifacts() []*AddArtifactsResponse_ArtifactSummary {\n\tif x != nil {\n\t\treturn x.Artifacts\n\t}\n\treturn nil\n}\n\n// Request to get current statuses of artifacts at the server side.\ntype ArtifactStatusesRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,5,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,3,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// The name of the artifact is expected in the form of a \"Relative Path\" that is made up of a\n\t// sequence of directories and the final file element.\n\t// Examples of \"Relative Path\"s: \"jars/test.jar\", \"classes/xyz.class\", \"abc.xyz\", \"a/b/X.jar\".\n\t// The server is expected to maintain the hierarchy of files as defined by their name. (i.e\n\t// The relative path of the file on the server's filesystem will be the same as the name of\n\t// the provided artifact)\n\tNames []string `protobuf:\"bytes,4,rep,name=names,proto3\" json:\"names,omitempty\"`\n}\n\nfunc (x *ArtifactStatusesRequest) Reset() {\n\t*x = ArtifactStatusesRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ArtifactStatusesRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ArtifactStatusesRequest) ProtoMessage() {}\n\nfunc (x *ArtifactStatusesRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ArtifactStatusesRequest.ProtoReflect.Descriptor instead.\nfunc (*ArtifactStatusesRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{11}\n}\n\nfunc (x *ArtifactStatusesRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ArtifactStatusesRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ArtifactStatusesRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ArtifactStatusesRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (x *ArtifactStatusesRequest) GetNames() []string {\n\tif x != nil {\n\t\treturn x.Names\n\t}\n\treturn nil\n}\n\n// Response to checking artifact statuses.\n// Next ID: 4\ntype ArtifactStatusesResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Session id in which the ArtifactStatus was running.\n\tSessionId string `protobuf:\"bytes,2,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// A map of artifact names to their statuses.\n\tStatuses map[string]*ArtifactStatusesResponse_ArtifactStatus `protobuf:\"bytes,1,rep,name=statuses,proto3\" json:\"statuses,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *ArtifactStatusesResponse) Reset() {\n\t*x = ArtifactStatusesResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ArtifactStatusesResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ArtifactStatusesResponse) ProtoMessage() {}\n\nfunc (x *ArtifactStatusesResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ArtifactStatusesResponse.ProtoReflect.Descriptor instead.\nfunc (*ArtifactStatusesResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{12}\n}\n\nfunc (x *ArtifactStatusesResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ArtifactStatusesResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ArtifactStatusesResponse) GetStatuses() map[string]*ArtifactStatusesResponse_ArtifactStatus {\n\tif x != nil {\n\t\treturn x.Statuses\n\t}\n\treturn nil\n}\n\ntype InterruptRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id specifies a spark session for a user id (which is specified\n\t// by user_context.user_id). The session_id is set by the client to be able to\n\t// collate streaming responses from different queries within the dedicated session.\n\t// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,3,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// (Required) The type of interrupt to execute.\n\tInterruptType InterruptRequest_InterruptType `protobuf:\"varint,4,opt,name=interrupt_type,json=interruptType,proto3,enum=spark.connect.InterruptRequest_InterruptType\" json:\"interrupt_type,omitempty\"`\n\t// Types that are assignable to Interrupt:\n\t//\n\t//\t*InterruptRequest_OperationTag\n\t//\t*InterruptRequest_OperationId\n\tInterrupt isInterruptRequest_Interrupt `protobuf_oneof:\"interrupt\"`\n}\n\nfunc (x *InterruptRequest) Reset() {\n\t*x = InterruptRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *InterruptRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*InterruptRequest) ProtoMessage() {}\n\nfunc (x *InterruptRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use InterruptRequest.ProtoReflect.Descriptor instead.\nfunc (*InterruptRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{13}\n}\n\nfunc (x *InterruptRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *InterruptRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptRequest) GetInterruptType() InterruptRequest_InterruptType {\n\tif x != nil {\n\t\treturn x.InterruptType\n\t}\n\treturn InterruptRequest_INTERRUPT_TYPE_UNSPECIFIED\n}\n\nfunc (m *InterruptRequest) GetInterrupt() isInterruptRequest_Interrupt {\n\tif m != nil {\n\t\treturn m.Interrupt\n\t}\n\treturn nil\n}\n\nfunc (x *InterruptRequest) GetOperationTag() string {\n\tif x, ok := x.GetInterrupt().(*InterruptRequest_OperationTag); ok {\n\t\treturn x.OperationTag\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptRequest) GetOperationId() string {\n\tif x, ok := x.GetInterrupt().(*InterruptRequest_OperationId); ok {\n\t\treturn x.OperationId\n\t}\n\treturn \"\"\n}\n\ntype isInterruptRequest_Interrupt interface {\n\tisInterruptRequest_Interrupt()\n}\n\ntype InterruptRequest_OperationTag struct {\n\t// if interrupt_tag == INTERRUPT_TYPE_TAG, interrupt operation with this tag.\n\tOperationTag string `protobuf:\"bytes,5,opt,name=operation_tag,json=operationTag,proto3,oneof\"`\n}\n\ntype InterruptRequest_OperationId struct {\n\t// if interrupt_tag == INTERRUPT_TYPE_OPERATION_ID, interrupt operation with this operation_id.\n\tOperationId string `protobuf:\"bytes,6,opt,name=operation_id,json=operationId,proto3,oneof\"`\n}\n\nfunc (*InterruptRequest_OperationTag) isInterruptRequest_Interrupt() {}\n\nfunc (*InterruptRequest_OperationId) isInterruptRequest_Interrupt() {}\n\n// Next ID: 4\ntype InterruptResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Session id in which the interrupt was running.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// Operation ids of the executions which were interrupted.\n\tInterruptedIds []string `protobuf:\"bytes,2,rep,name=interrupted_ids,json=interruptedIds,proto3\" json:\"interrupted_ids,omitempty\"`\n}\n\nfunc (x *InterruptResponse) Reset() {\n\t*x = InterruptResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *InterruptResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*InterruptResponse) ProtoMessage() {}\n\nfunc (x *InterruptResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use InterruptResponse.ProtoReflect.Descriptor instead.\nfunc (*InterruptResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{14}\n}\n\nfunc (x *InterruptResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *InterruptResponse) GetInterruptedIds() []string {\n\tif x != nil {\n\t\treturn x.InterruptedIds\n\t}\n\treturn nil\n}\n\ntype ReattachOptions struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// If true, the request can be reattached to using ReattachExecute.\n\t// ReattachExecute can be used either if the stream broke with a GRPC network error,\n\t// or if the server closed the stream without sending a response with StreamStatus.complete=true.\n\t// The server will keep a buffer of responses in case a response is lost, and\n\t// ReattachExecute needs to back-track.\n\t//\n\t// If false, the execution response stream will will not be reattachable, and all responses are\n\t// immediately released by the server after being sent.\n\tReattachable bool `protobuf:\"varint,1,opt,name=reattachable,proto3\" json:\"reattachable,omitempty\"`\n}\n\nfunc (x *ReattachOptions) Reset() {\n\t*x = ReattachOptions{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReattachOptions) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReattachOptions) ProtoMessage() {}\n\nfunc (x *ReattachOptions) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReattachOptions.ProtoReflect.Descriptor instead.\nfunc (*ReattachOptions) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{15}\n}\n\nfunc (x *ReattachOptions) GetReattachable() bool {\n\tif x != nil {\n\t\treturn x.Reattachable\n\t}\n\treturn false\n}\n\ntype ReattachExecuteRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id of the request to reattach to.\n\t// This must be an id of existing session.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,6,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\t//\n\t// user_context.user_id and session+id both identify a unique remote spark session on the\n\t// server side.\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Required)\n\t// Provide an id of the request to reattach to.\n\t// This must be an id of existing operation.\n\tOperationId string `protobuf:\"bytes,3,opt,name=operation_id,json=operationId,proto3\" json:\"operation_id,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,4,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// (Optional)\n\t// Last already processed response id from the response stream.\n\t// After reattach, server will resume the response stream after that response.\n\t// If not specified, server will restart the stream from the start.\n\t//\n\t// Note: server controls the amount of responses that it buffers and it may drop responses,\n\t// that are far behind the latest returned response, so this can't be used to arbitrarily\n\t// scroll back the cursor. If the response is no longer available, this will result in an error.\n\tLastResponseId *string `protobuf:\"bytes,5,opt,name=last_response_id,json=lastResponseId,proto3,oneof\" json:\"last_response_id,omitempty\"`\n}\n\nfunc (x *ReattachExecuteRequest) Reset() {\n\t*x = ReattachExecuteRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReattachExecuteRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReattachExecuteRequest) ProtoMessage() {}\n\nfunc (x *ReattachExecuteRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReattachExecuteRequest.ProtoReflect.Descriptor instead.\nfunc (*ReattachExecuteRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{16}\n}\n\nfunc (x *ReattachExecuteRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReattachExecuteRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReattachExecuteRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ReattachExecuteRequest) GetOperationId() string {\n\tif x != nil {\n\t\treturn x.OperationId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReattachExecuteRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReattachExecuteRequest) GetLastResponseId() string {\n\tif x != nil && x.LastResponseId != nil {\n\t\treturn *x.LastResponseId\n\t}\n\treturn \"\"\n}\n\ntype ReleaseExecuteRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id of the request to reattach to.\n\t// This must be an id of existing session.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// (Required) User context\n\t//\n\t// user_context.user_id and session+id both identify a unique remote spark session on the\n\t// server side.\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Required)\n\t// Provide an id of the request to reattach to.\n\t// This must be an id of existing operation.\n\tOperationId string `protobuf:\"bytes,3,opt,name=operation_id,json=operationId,proto3\" json:\"operation_id,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,4,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// Types that are assignable to Release:\n\t//\n\t//\t*ReleaseExecuteRequest_ReleaseAll_\n\t//\t*ReleaseExecuteRequest_ReleaseUntil_\n\tRelease isReleaseExecuteRequest_Release `protobuf_oneof:\"release\"`\n}\n\nfunc (x *ReleaseExecuteRequest) Reset() {\n\t*x = ReleaseExecuteRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseExecuteRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseExecuteRequest) ProtoMessage() {}\n\nfunc (x *ReleaseExecuteRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseExecuteRequest.ProtoReflect.Descriptor instead.\nfunc (*ReleaseExecuteRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{17}\n}\n\nfunc (x *ReleaseExecuteRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseExecuteRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseExecuteRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ReleaseExecuteRequest) GetOperationId() string {\n\tif x != nil {\n\t\treturn x.OperationId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseExecuteRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (m *ReleaseExecuteRequest) GetRelease() isReleaseExecuteRequest_Release {\n\tif m != nil {\n\t\treturn m.Release\n\t}\n\treturn nil\n}\n\nfunc (x *ReleaseExecuteRequest) GetReleaseAll() *ReleaseExecuteRequest_ReleaseAll {\n\tif x, ok := x.GetRelease().(*ReleaseExecuteRequest_ReleaseAll_); ok {\n\t\treturn x.ReleaseAll\n\t}\n\treturn nil\n}\n\nfunc (x *ReleaseExecuteRequest) GetReleaseUntil() *ReleaseExecuteRequest_ReleaseUntil {\n\tif x, ok := x.GetRelease().(*ReleaseExecuteRequest_ReleaseUntil_); ok {\n\t\treturn x.ReleaseUntil\n\t}\n\treturn nil\n}\n\ntype isReleaseExecuteRequest_Release interface {\n\tisReleaseExecuteRequest_Release()\n}\n\ntype ReleaseExecuteRequest_ReleaseAll_ struct {\n\tReleaseAll *ReleaseExecuteRequest_ReleaseAll `protobuf:\"bytes,5,opt,name=release_all,json=releaseAll,proto3,oneof\"`\n}\n\ntype ReleaseExecuteRequest_ReleaseUntil_ struct {\n\tReleaseUntil *ReleaseExecuteRequest_ReleaseUntil `protobuf:\"bytes,6,opt,name=release_until,json=releaseUntil,proto3,oneof\"`\n}\n\nfunc (*ReleaseExecuteRequest_ReleaseAll_) isReleaseExecuteRequest_Release() {}\n\nfunc (*ReleaseExecuteRequest_ReleaseUntil_) isReleaseExecuteRequest_Release() {}\n\n// Next ID: 4\ntype ReleaseExecuteResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Session id in which the release was running.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\t// Operation id of the operation on which the release executed.\n\t// If the operation couldn't be found (because e.g. it was concurrently released), will be unset.\n\t// Otherwise, it will be equal to the operation_id from request.\n\tOperationId *string `protobuf:\"bytes,2,opt,name=operation_id,json=operationId,proto3,oneof\" json:\"operation_id,omitempty\"`\n}\n\nfunc (x *ReleaseExecuteResponse) Reset() {\n\t*x = ReleaseExecuteResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseExecuteResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseExecuteResponse) ProtoMessage() {}\n\nfunc (x *ReleaseExecuteResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseExecuteResponse.ProtoReflect.Descriptor instead.\nfunc (*ReleaseExecuteResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{18}\n}\n\nfunc (x *ReleaseExecuteResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseExecuteResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseExecuteResponse) GetOperationId() string {\n\tif x != nil && x.OperationId != nil {\n\t\treturn *x.OperationId\n\t}\n\treturn \"\"\n}\n\ntype ReleaseSessionRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t//\n\t// The session_id of the request to reattach to.\n\t// This must be an id of existing session.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Required) User context\n\t//\n\t// user_context.user_id and session+id both identify a unique remote spark session on the\n\t// server side.\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,3,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n\t// Signals the server to allow the client to reconnect to the session after it is released.\n\t//\n\t// By default, the server tombstones the session upon release, preventing reconnections and\n\t// fully cleaning the session state.\n\t//\n\t// If this flag is set to true, the server may permit the client to reconnect to the session\n\t// post-release, even if the session state has been cleaned. This can result in missing state,\n\t// such as Temporary Views, Temporary UDFs, or the Current Catalog, in the reconnected session.\n\t//\n\t// Use this option sparingly and only when the client fully understands the implications of\n\t// reconnecting to a released session. The client must ensure that any queries executed do not\n\t// rely on the session state prior to its release.\n\tAllowReconnect bool `protobuf:\"varint,4,opt,name=allow_reconnect,json=allowReconnect,proto3\" json:\"allow_reconnect,omitempty\"`\n}\n\nfunc (x *ReleaseSessionRequest) Reset() {\n\t*x = ReleaseSessionRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseSessionRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseSessionRequest) ProtoMessage() {}\n\nfunc (x *ReleaseSessionRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseSessionRequest.ProtoReflect.Descriptor instead.\nfunc (*ReleaseSessionRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{19}\n}\n\nfunc (x *ReleaseSessionRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseSessionRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *ReleaseSessionRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseSessionRequest) GetAllowReconnect() bool {\n\tif x != nil {\n\t\treturn x.AllowReconnect\n\t}\n\treturn false\n}\n\n// Next ID: 3\ntype ReleaseSessionResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Session id of the session on which the release executed.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,2,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n}\n\nfunc (x *ReleaseSessionResponse) Reset() {\n\t*x = ReleaseSessionResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseSessionResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseSessionResponse) ProtoMessage() {}\n\nfunc (x *ReleaseSessionResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseSessionResponse.ProtoReflect.Descriptor instead.\nfunc (*ReleaseSessionResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{20}\n}\n\nfunc (x *ReleaseSessionResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *ReleaseSessionResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\ntype FetchErrorDetailsRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\t// The session_id specifies a Spark session for a user identified by user_context.user_id.\n\t// The id should be a UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`.\n\tSessionId string `protobuf:\"bytes,1,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// (Optional)\n\t//\n\t// Server-side generated idempotency key from the previous responses (if any). Server\n\t// can use this to validate that the server side session has not changed.\n\tClientObservedServerSideSessionId *string `protobuf:\"bytes,5,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof\" json:\"client_observed_server_side_session_id,omitempty\"`\n\t// User context\n\tUserContext *UserContext `protobuf:\"bytes,2,opt,name=user_context,json=userContext,proto3\" json:\"user_context,omitempty\"`\n\t// (Required)\n\t// The id of the error.\n\tErrorId string `protobuf:\"bytes,3,opt,name=error_id,json=errorId,proto3\" json:\"error_id,omitempty\"`\n\t// Provides optional information about the client sending the request. This field\n\t// can be used for language or version specific information and is only intended for\n\t// logging purposes and will not be interpreted by the server.\n\tClientType *string `protobuf:\"bytes,4,opt,name=client_type,json=clientType,proto3,oneof\" json:\"client_type,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsRequest) Reset() {\n\t*x = FetchErrorDetailsRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsRequest) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsRequest.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{21}\n}\n\nfunc (x *FetchErrorDetailsRequest) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsRequest) GetClientObservedServerSideSessionId() string {\n\tif x != nil && x.ClientObservedServerSideSessionId != nil {\n\t\treturn *x.ClientObservedServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsRequest) GetUserContext() *UserContext {\n\tif x != nil {\n\t\treturn x.UserContext\n\t}\n\treturn nil\n}\n\nfunc (x *FetchErrorDetailsRequest) GetErrorId() string {\n\tif x != nil {\n\t\treturn x.ErrorId\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsRequest) GetClientType() string {\n\tif x != nil && x.ClientType != nil {\n\t\treturn *x.ClientType\n\t}\n\treturn \"\"\n}\n\n// Next ID: 5\ntype FetchErrorDetailsResponse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Server-side generated idempotency key that the client can use to assert that the server side\n\t// session has not changed.\n\tServerSideSessionId string `protobuf:\"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3\" json:\"server_side_session_id,omitempty\"`\n\tSessionId           string `protobuf:\"bytes,4,opt,name=session_id,json=sessionId,proto3\" json:\"session_id,omitempty\"`\n\t// The index of the root error in errors. The field will not be set if the error is not found.\n\tRootErrorIdx *int32 `protobuf:\"varint,1,opt,name=root_error_idx,json=rootErrorIdx,proto3,oneof\" json:\"root_error_idx,omitempty\"`\n\t// A list of errors.\n\tErrors []*FetchErrorDetailsResponse_Error `protobuf:\"bytes,2,rep,name=errors,proto3\" json:\"errors,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsResponse) Reset() {\n\t*x = FetchErrorDetailsResponse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsResponse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsResponse) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsResponse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsResponse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22}\n}\n\nfunc (x *FetchErrorDetailsResponse) GetServerSideSessionId() string {\n\tif x != nil {\n\t\treturn x.ServerSideSessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse) GetSessionId() string {\n\tif x != nil {\n\t\treturn x.SessionId\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse) GetRootErrorIdx() int32 {\n\tif x != nil && x.RootErrorIdx != nil {\n\t\treturn *x.RootErrorIdx\n\t}\n\treturn 0\n}\n\nfunc (x *FetchErrorDetailsResponse) GetErrors() []*FetchErrorDetailsResponse_Error {\n\tif x != nil {\n\t\treturn x.Errors\n\t}\n\treturn nil\n}\n\ntype CheckpointCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan checkpointed.\n\tRelation *CachedRemoteRelation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n}\n\nfunc (x *CheckpointCommandResult) Reset() {\n\t*x = CheckpointCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CheckpointCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CheckpointCommandResult) ProtoMessage() {}\n\nfunc (x *CheckpointCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CheckpointCommandResult.ProtoReflect.Descriptor instead.\nfunc (*CheckpointCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{23}\n}\n\nfunc (x *CheckpointCommandResult) GetRelation() *CachedRemoteRelation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_Schema struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_Schema) Reset() {\n\t*x = AnalyzePlanRequest_Schema{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[24]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_Schema) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_Schema) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_Schema) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[24]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_Schema.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_Schema) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 0}\n}\n\nfunc (x *AnalyzePlanRequest_Schema) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\n// Explains the input plan based on a configurable mode.\ntype AnalyzePlanRequest_Explain struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n\t// (Required) For analyzePlan rpc calls, configure the mode to explain plan in strings.\n\tExplainMode AnalyzePlanRequest_Explain_ExplainMode `protobuf:\"varint,2,opt,name=explain_mode,json=explainMode,proto3,enum=spark.connect.AnalyzePlanRequest_Explain_ExplainMode\" json:\"explain_mode,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_Explain) Reset() {\n\t*x = AnalyzePlanRequest_Explain{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[25]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_Explain) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_Explain) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_Explain) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[25]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_Explain.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_Explain) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 1}\n}\n\nfunc (x *AnalyzePlanRequest_Explain) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest_Explain) GetExplainMode() AnalyzePlanRequest_Explain_ExplainMode {\n\tif x != nil {\n\t\treturn x.ExplainMode\n\t}\n\treturn AnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED\n}\n\ntype AnalyzePlanRequest_TreeString struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n\t// (Optional) Max level of the schema.\n\tLevel *int32 `protobuf:\"varint,2,opt,name=level,proto3,oneof\" json:\"level,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_TreeString) Reset() {\n\t*x = AnalyzePlanRequest_TreeString{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[26]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_TreeString) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_TreeString) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_TreeString) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[26]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_TreeString.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_TreeString) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 2}\n}\n\nfunc (x *AnalyzePlanRequest_TreeString) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest_TreeString) GetLevel() int32 {\n\tif x != nil && x.Level != nil {\n\t\treturn *x.Level\n\t}\n\treturn 0\n}\n\ntype AnalyzePlanRequest_IsLocal struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_IsLocal) Reset() {\n\t*x = AnalyzePlanRequest_IsLocal{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[27]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_IsLocal) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_IsLocal) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_IsLocal) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[27]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_IsLocal.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_IsLocal) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 3}\n}\n\nfunc (x *AnalyzePlanRequest_IsLocal) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_IsStreaming struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_IsStreaming) Reset() {\n\t*x = AnalyzePlanRequest_IsStreaming{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[28]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_IsStreaming) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_IsStreaming) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_IsStreaming) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[28]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_IsStreaming.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_IsStreaming) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 4}\n}\n\nfunc (x *AnalyzePlanRequest_IsStreaming) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_InputFiles struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to be analyzed.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_InputFiles) Reset() {\n\t*x = AnalyzePlanRequest_InputFiles{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[29]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_InputFiles) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_InputFiles) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_InputFiles) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[29]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_InputFiles.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_InputFiles) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 5}\n}\n\nfunc (x *AnalyzePlanRequest_InputFiles) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_SparkVersion struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *AnalyzePlanRequest_SparkVersion) Reset() {\n\t*x = AnalyzePlanRequest_SparkVersion{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[30]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_SparkVersion) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_SparkVersion) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_SparkVersion) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[30]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_SparkVersion.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_SparkVersion) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 6}\n}\n\ntype AnalyzePlanRequest_DDLParse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The DDL formatted string to be parsed.\n\tDdlString string `protobuf:\"bytes,1,opt,name=ddl_string,json=ddlString,proto3\" json:\"ddl_string,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_DDLParse) Reset() {\n\t*x = AnalyzePlanRequest_DDLParse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[31]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_DDLParse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_DDLParse) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_DDLParse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[31]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_DDLParse.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_DDLParse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 7}\n}\n\nfunc (x *AnalyzePlanRequest_DDLParse) GetDdlString() string {\n\tif x != nil {\n\t\treturn x.DdlString\n\t}\n\treturn \"\"\n}\n\n// Returns `true` when the logical query plans  are equal and therefore return same results.\ntype AnalyzePlanRequest_SameSemantics struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The plan to be compared.\n\tTargetPlan *Plan `protobuf:\"bytes,1,opt,name=target_plan,json=targetPlan,proto3\" json:\"target_plan,omitempty\"`\n\t// (Required) The other plan to be compared.\n\tOtherPlan *Plan `protobuf:\"bytes,2,opt,name=other_plan,json=otherPlan,proto3\" json:\"other_plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_SameSemantics) Reset() {\n\t*x = AnalyzePlanRequest_SameSemantics{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[32]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_SameSemantics) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_SameSemantics) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_SameSemantics) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[32]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_SameSemantics.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_SameSemantics) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 8}\n}\n\nfunc (x *AnalyzePlanRequest_SameSemantics) GetTargetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.TargetPlan\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest_SameSemantics) GetOtherPlan() *Plan {\n\tif x != nil {\n\t\treturn x.OtherPlan\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_SemanticHash struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to get a hashCode.\n\tPlan *Plan `protobuf:\"bytes,1,opt,name=plan,proto3\" json:\"plan,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_SemanticHash) Reset() {\n\t*x = AnalyzePlanRequest_SemanticHash{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[33]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_SemanticHash) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_SemanticHash) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_SemanticHash) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[33]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_SemanticHash.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_SemanticHash) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 9}\n}\n\nfunc (x *AnalyzePlanRequest_SemanticHash) GetPlan() *Plan {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_Persist struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to persist.\n\tRelation *Relation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n\t// (Optional) The storage level.\n\tStorageLevel *StorageLevel `protobuf:\"bytes,2,opt,name=storage_level,json=storageLevel,proto3,oneof\" json:\"storage_level,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_Persist) Reset() {\n\t*x = AnalyzePlanRequest_Persist{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[34]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_Persist) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_Persist) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_Persist) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[34]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_Persist.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_Persist) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 10}\n}\n\nfunc (x *AnalyzePlanRequest_Persist) GetRelation() *Relation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest_Persist) GetStorageLevel() *StorageLevel {\n\tif x != nil {\n\t\treturn x.StorageLevel\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_Unpersist struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to unpersist.\n\tRelation *Relation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n\t// (Optional) Whether to block until all blocks are deleted.\n\tBlocking *bool `protobuf:\"varint,2,opt,name=blocking,proto3,oneof\" json:\"blocking,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_Unpersist) Reset() {\n\t*x = AnalyzePlanRequest_Unpersist{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[35]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_Unpersist) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_Unpersist) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_Unpersist) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[35]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_Unpersist.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_Unpersist) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 11}\n}\n\nfunc (x *AnalyzePlanRequest_Unpersist) GetRelation() *Relation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\nfunc (x *AnalyzePlanRequest_Unpersist) GetBlocking() bool {\n\tif x != nil && x.Blocking != nil {\n\t\treturn *x.Blocking\n\t}\n\treturn false\n}\n\ntype AnalyzePlanRequest_GetStorageLevel struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to get the storage level.\n\tRelation *Relation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_GetStorageLevel) Reset() {\n\t*x = AnalyzePlanRequest_GetStorageLevel{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[36]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_GetStorageLevel) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_GetStorageLevel) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_GetStorageLevel) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[36]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_GetStorageLevel.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_GetStorageLevel) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 12}\n}\n\nfunc (x *AnalyzePlanRequest_GetStorageLevel) GetRelation() *Relation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanRequest_JsonToDDL struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The JSON formatted string to be converted to DDL.\n\tJsonString string `protobuf:\"bytes,1,opt,name=json_string,json=jsonString,proto3\" json:\"json_string,omitempty\"`\n}\n\nfunc (x *AnalyzePlanRequest_JsonToDDL) Reset() {\n\t*x = AnalyzePlanRequest_JsonToDDL{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[37]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanRequest_JsonToDDL) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanRequest_JsonToDDL) ProtoMessage() {}\n\nfunc (x *AnalyzePlanRequest_JsonToDDL) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[37]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanRequest_JsonToDDL.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanRequest_JsonToDDL) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{2, 13}\n}\n\nfunc (x *AnalyzePlanRequest_JsonToDDL) GetJsonString() string {\n\tif x != nil {\n\t\treturn x.JsonString\n\t}\n\treturn \"\"\n}\n\ntype AnalyzePlanResponse_Schema struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tSchema *DataType `protobuf:\"bytes,1,opt,name=schema,proto3\" json:\"schema,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_Schema) Reset() {\n\t*x = AnalyzePlanResponse_Schema{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[38]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_Schema) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_Schema) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_Schema) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[38]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_Schema.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_Schema) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 0}\n}\n\nfunc (x *AnalyzePlanResponse_Schema) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanResponse_Explain struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tExplainString string `protobuf:\"bytes,1,opt,name=explain_string,json=explainString,proto3\" json:\"explain_string,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_Explain) Reset() {\n\t*x = AnalyzePlanResponse_Explain{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[39]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_Explain) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_Explain) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_Explain) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[39]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_Explain.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_Explain) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 1}\n}\n\nfunc (x *AnalyzePlanResponse_Explain) GetExplainString() string {\n\tif x != nil {\n\t\treturn x.ExplainString\n\t}\n\treturn \"\"\n}\n\ntype AnalyzePlanResponse_TreeString struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTreeString string `protobuf:\"bytes,1,opt,name=tree_string,json=treeString,proto3\" json:\"tree_string,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_TreeString) Reset() {\n\t*x = AnalyzePlanResponse_TreeString{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[40]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_TreeString) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_TreeString) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_TreeString) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[40]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_TreeString.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_TreeString) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 2}\n}\n\nfunc (x *AnalyzePlanResponse_TreeString) GetTreeString() string {\n\tif x != nil {\n\t\treturn x.TreeString\n\t}\n\treturn \"\"\n}\n\ntype AnalyzePlanResponse_IsLocal struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tIsLocal bool `protobuf:\"varint,1,opt,name=is_local,json=isLocal,proto3\" json:\"is_local,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_IsLocal) Reset() {\n\t*x = AnalyzePlanResponse_IsLocal{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[41]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_IsLocal) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_IsLocal) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_IsLocal) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[41]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_IsLocal.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_IsLocal) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 3}\n}\n\nfunc (x *AnalyzePlanResponse_IsLocal) GetIsLocal() bool {\n\tif x != nil {\n\t\treturn x.IsLocal\n\t}\n\treturn false\n}\n\ntype AnalyzePlanResponse_IsStreaming struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tIsStreaming bool `protobuf:\"varint,1,opt,name=is_streaming,json=isStreaming,proto3\" json:\"is_streaming,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_IsStreaming) Reset() {\n\t*x = AnalyzePlanResponse_IsStreaming{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[42]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_IsStreaming) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_IsStreaming) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_IsStreaming) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[42]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_IsStreaming.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_IsStreaming) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 4}\n}\n\nfunc (x *AnalyzePlanResponse_IsStreaming) GetIsStreaming() bool {\n\tif x != nil {\n\t\treturn x.IsStreaming\n\t}\n\treturn false\n}\n\ntype AnalyzePlanResponse_InputFiles struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// A best-effort snapshot of the files that compose this Dataset\n\tFiles []string `protobuf:\"bytes,1,rep,name=files,proto3\" json:\"files,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_InputFiles) Reset() {\n\t*x = AnalyzePlanResponse_InputFiles{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[43]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_InputFiles) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_InputFiles) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_InputFiles) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[43]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_InputFiles.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_InputFiles) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 5}\n}\n\nfunc (x *AnalyzePlanResponse_InputFiles) GetFiles() []string {\n\tif x != nil {\n\t\treturn x.Files\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanResponse_SparkVersion struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tVersion string `protobuf:\"bytes,1,opt,name=version,proto3\" json:\"version,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_SparkVersion) Reset() {\n\t*x = AnalyzePlanResponse_SparkVersion{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[44]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_SparkVersion) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_SparkVersion) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_SparkVersion) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[44]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_SparkVersion.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_SparkVersion) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 6}\n}\n\nfunc (x *AnalyzePlanResponse_SparkVersion) GetVersion() string {\n\tif x != nil {\n\t\treturn x.Version\n\t}\n\treturn \"\"\n}\n\ntype AnalyzePlanResponse_DDLParse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tParsed *DataType `protobuf:\"bytes,1,opt,name=parsed,proto3\" json:\"parsed,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_DDLParse) Reset() {\n\t*x = AnalyzePlanResponse_DDLParse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[45]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_DDLParse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_DDLParse) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_DDLParse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[45]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_DDLParse.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_DDLParse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 7}\n}\n\nfunc (x *AnalyzePlanResponse_DDLParse) GetParsed() *DataType {\n\tif x != nil {\n\t\treturn x.Parsed\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanResponse_SameSemantics struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tResult bool `protobuf:\"varint,1,opt,name=result,proto3\" json:\"result,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_SameSemantics) Reset() {\n\t*x = AnalyzePlanResponse_SameSemantics{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[46]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_SameSemantics) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_SameSemantics) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_SameSemantics) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[46]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_SameSemantics.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_SameSemantics) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 8}\n}\n\nfunc (x *AnalyzePlanResponse_SameSemantics) GetResult() bool {\n\tif x != nil {\n\t\treturn x.Result\n\t}\n\treturn false\n}\n\ntype AnalyzePlanResponse_SemanticHash struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tResult int32 `protobuf:\"varint,1,opt,name=result,proto3\" json:\"result,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_SemanticHash) Reset() {\n\t*x = AnalyzePlanResponse_SemanticHash{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[47]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_SemanticHash) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_SemanticHash) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_SemanticHash) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[47]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_SemanticHash.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_SemanticHash) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 9}\n}\n\nfunc (x *AnalyzePlanResponse_SemanticHash) GetResult() int32 {\n\tif x != nil {\n\t\treturn x.Result\n\t}\n\treturn 0\n}\n\ntype AnalyzePlanResponse_Persist struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *AnalyzePlanResponse_Persist) Reset() {\n\t*x = AnalyzePlanResponse_Persist{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[48]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_Persist) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_Persist) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_Persist) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[48]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_Persist.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_Persist) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 10}\n}\n\ntype AnalyzePlanResponse_Unpersist struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *AnalyzePlanResponse_Unpersist) Reset() {\n\t*x = AnalyzePlanResponse_Unpersist{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[49]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_Unpersist) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_Unpersist) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_Unpersist) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[49]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_Unpersist.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_Unpersist) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 11}\n}\n\ntype AnalyzePlanResponse_GetStorageLevel struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The StorageLevel as a result of get_storage_level request.\n\tStorageLevel *StorageLevel `protobuf:\"bytes,1,opt,name=storage_level,json=storageLevel,proto3\" json:\"storage_level,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_GetStorageLevel) Reset() {\n\t*x = AnalyzePlanResponse_GetStorageLevel{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[50]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_GetStorageLevel) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_GetStorageLevel) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_GetStorageLevel) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[50]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_GetStorageLevel.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_GetStorageLevel) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 12}\n}\n\nfunc (x *AnalyzePlanResponse_GetStorageLevel) GetStorageLevel() *StorageLevel {\n\tif x != nil {\n\t\treturn x.StorageLevel\n\t}\n\treturn nil\n}\n\ntype AnalyzePlanResponse_JsonToDDL struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tDdlString string `protobuf:\"bytes,1,opt,name=ddl_string,json=ddlString,proto3\" json:\"ddl_string,omitempty\"`\n}\n\nfunc (x *AnalyzePlanResponse_JsonToDDL) Reset() {\n\t*x = AnalyzePlanResponse_JsonToDDL{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[51]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AnalyzePlanResponse_JsonToDDL) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AnalyzePlanResponse_JsonToDDL) ProtoMessage() {}\n\nfunc (x *AnalyzePlanResponse_JsonToDDL) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[51]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AnalyzePlanResponse_JsonToDDL.ProtoReflect.Descriptor instead.\nfunc (*AnalyzePlanResponse_JsonToDDL) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{3, 13}\n}\n\nfunc (x *AnalyzePlanResponse_JsonToDDL) GetDdlString() string {\n\tif x != nil {\n\t\treturn x.DdlString\n\t}\n\treturn \"\"\n}\n\ntype ExecutePlanRequest_RequestOption struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to RequestOption:\n\t//\n\t//\t*ExecutePlanRequest_RequestOption_ReattachOptions\n\t//\t*ExecutePlanRequest_RequestOption_Extension\n\tRequestOption isExecutePlanRequest_RequestOption_RequestOption `protobuf_oneof:\"request_option\"`\n}\n\nfunc (x *ExecutePlanRequest_RequestOption) Reset() {\n\t*x = ExecutePlanRequest_RequestOption{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[52]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanRequest_RequestOption) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanRequest_RequestOption) ProtoMessage() {}\n\nfunc (x *ExecutePlanRequest_RequestOption) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[52]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanRequest_RequestOption.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanRequest_RequestOption) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{4, 0}\n}\n\nfunc (m *ExecutePlanRequest_RequestOption) GetRequestOption() isExecutePlanRequest_RequestOption_RequestOption {\n\tif m != nil {\n\t\treturn m.RequestOption\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanRequest_RequestOption) GetReattachOptions() *ReattachOptions {\n\tif x, ok := x.GetRequestOption().(*ExecutePlanRequest_RequestOption_ReattachOptions); ok {\n\t\treturn x.ReattachOptions\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanRequest_RequestOption) GetExtension() *anypb.Any {\n\tif x, ok := x.GetRequestOption().(*ExecutePlanRequest_RequestOption_Extension); ok {\n\t\treturn x.Extension\n\t}\n\treturn nil\n}\n\ntype isExecutePlanRequest_RequestOption_RequestOption interface {\n\tisExecutePlanRequest_RequestOption_RequestOption()\n}\n\ntype ExecutePlanRequest_RequestOption_ReattachOptions struct {\n\tReattachOptions *ReattachOptions `protobuf:\"bytes,1,opt,name=reattach_options,json=reattachOptions,proto3,oneof\"`\n}\n\ntype ExecutePlanRequest_RequestOption_Extension struct {\n\t// Extension type for request options\n\tExtension *anypb.Any `protobuf:\"bytes,999,opt,name=extension,proto3,oneof\"`\n}\n\nfunc (*ExecutePlanRequest_RequestOption_ReattachOptions) isExecutePlanRequest_RequestOption_RequestOption() {\n}\n\nfunc (*ExecutePlanRequest_RequestOption_Extension) isExecutePlanRequest_RequestOption_RequestOption() {\n}\n\n// A SQL command returns an opaque Relation that can be directly used as input for the next\n// call.\ntype ExecutePlanResponse_SqlCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tRelation *Relation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_SqlCommandResult) Reset() {\n\t*x = ExecutePlanResponse_SqlCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[53]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_SqlCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_SqlCommandResult) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_SqlCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[53]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_SqlCommandResult.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_SqlCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 0}\n}\n\nfunc (x *ExecutePlanResponse_SqlCommandResult) GetRelation() *Relation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\n// Batch results of metrics.\ntype ExecutePlanResponse_ArrowBatch struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Count rows in `data`. Must match the number of rows inside `data`.\n\tRowCount int64 `protobuf:\"varint,1,opt,name=row_count,json=rowCount,proto3\" json:\"row_count,omitempty\"`\n\t// Serialized Arrow data.\n\tData []byte `protobuf:\"bytes,2,opt,name=data,proto3\" json:\"data,omitempty\"`\n\t// If set, row offset of the start of this ArrowBatch in execution results.\n\tStartOffset *int64 `protobuf:\"varint,3,opt,name=start_offset,json=startOffset,proto3,oneof\" json:\"start_offset,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) Reset() {\n\t*x = ExecutePlanResponse_ArrowBatch{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[54]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_ArrowBatch) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[54]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_ArrowBatch.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_ArrowBatch) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 1}\n}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) GetRowCount() int64 {\n\tif x != nil {\n\t\treturn x.RowCount\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) GetData() []byte {\n\tif x != nil {\n\t\treturn x.Data\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse_ArrowBatch) GetStartOffset() int64 {\n\tif x != nil && x.StartOffset != nil {\n\t\treturn *x.StartOffset\n\t}\n\treturn 0\n}\n\ntype ExecutePlanResponse_Metrics struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tMetrics []*ExecutePlanResponse_Metrics_MetricObject `protobuf:\"bytes,1,rep,name=metrics,proto3\" json:\"metrics,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_Metrics) Reset() {\n\t*x = ExecutePlanResponse_Metrics{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[55]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_Metrics) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_Metrics) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_Metrics) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[55]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_Metrics.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_Metrics) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 2}\n}\n\nfunc (x *ExecutePlanResponse_Metrics) GetMetrics() []*ExecutePlanResponse_Metrics_MetricObject {\n\tif x != nil {\n\t\treturn x.Metrics\n\t}\n\treturn nil\n}\n\ntype ExecutePlanResponse_ObservedMetrics struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tName   string                `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\tValues []*Expression_Literal `protobuf:\"bytes,2,rep,name=values,proto3\" json:\"values,omitempty\"`\n\tKeys   []string              `protobuf:\"bytes,3,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n\tPlanId int64                 `protobuf:\"varint,4,opt,name=plan_id,json=planId,proto3\" json:\"plan_id,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) Reset() {\n\t*x = ExecutePlanResponse_ObservedMetrics{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[56]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_ObservedMetrics) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[56]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_ObservedMetrics.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_ObservedMetrics) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 3}\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) GetValues() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) GetKeys() []string {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse_ObservedMetrics) GetPlanId() int64 {\n\tif x != nil {\n\t\treturn x.PlanId\n\t}\n\treturn 0\n}\n\ntype ExecutePlanResponse_ResultComplete struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *ExecutePlanResponse_ResultComplete) Reset() {\n\t*x = ExecutePlanResponse_ResultComplete{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[57]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_ResultComplete) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_ResultComplete) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_ResultComplete) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[57]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_ResultComplete.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_ResultComplete) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 4}\n}\n\n// This message is used to communicate progress about the query progress during the execution.\ntype ExecutePlanResponse_ExecutionProgress struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Captures the progress of each individual stage.\n\tStages []*ExecutePlanResponse_ExecutionProgress_StageInfo `protobuf:\"bytes,1,rep,name=stages,proto3\" json:\"stages,omitempty\"`\n\t// Captures the currently in progress tasks.\n\tNumInflightTasks int64 `protobuf:\"varint,2,opt,name=num_inflight_tasks,json=numInflightTasks,proto3\" json:\"num_inflight_tasks,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress) Reset() {\n\t*x = ExecutePlanResponse_ExecutionProgress{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[58]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_ExecutionProgress) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[58]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_ExecutionProgress.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_ExecutionProgress) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 5}\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress) GetStages() []*ExecutePlanResponse_ExecutionProgress_StageInfo {\n\tif x != nil {\n\t\treturn x.Stages\n\t}\n\treturn nil\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress) GetNumInflightTasks() int64 {\n\tif x != nil {\n\t\treturn x.NumInflightTasks\n\t}\n\treturn 0\n}\n\ntype ExecutePlanResponse_Metrics_MetricObject struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tName             string                                              `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\tPlanId           int64                                               `protobuf:\"varint,2,opt,name=plan_id,json=planId,proto3\" json:\"plan_id,omitempty\"`\n\tParent           int64                                               `protobuf:\"varint,3,opt,name=parent,proto3\" json:\"parent,omitempty\"`\n\tExecutionMetrics map[string]*ExecutePlanResponse_Metrics_MetricValue `protobuf:\"bytes,4,rep,name=execution_metrics,json=executionMetrics,proto3\" json:\"execution_metrics,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) Reset() {\n\t*x = ExecutePlanResponse_Metrics_MetricObject{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[59]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_Metrics_MetricObject) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[59]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_Metrics_MetricObject.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_Metrics_MetricObject) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 2, 0}\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) GetPlanId() int64 {\n\tif x != nil {\n\t\treturn x.PlanId\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) GetParent() int64 {\n\tif x != nil {\n\t\treturn x.Parent\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricObject) GetExecutionMetrics() map[string]*ExecutePlanResponse_Metrics_MetricValue {\n\tif x != nil {\n\t\treturn x.ExecutionMetrics\n\t}\n\treturn nil\n}\n\ntype ExecutePlanResponse_Metrics_MetricValue struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tName       string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\tValue      int64  `protobuf:\"varint,2,opt,name=value,proto3\" json:\"value,omitempty\"`\n\tMetricType string `protobuf:\"bytes,3,opt,name=metric_type,json=metricType,proto3\" json:\"metric_type,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) Reset() {\n\t*x = ExecutePlanResponse_Metrics_MetricValue{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[60]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_Metrics_MetricValue) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[60]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_Metrics_MetricValue.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_Metrics_MetricValue) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 2, 1}\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) GetValue() int64 {\n\tif x != nil {\n\t\treturn x.Value\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_Metrics_MetricValue) GetMetricType() string {\n\tif x != nil {\n\t\treturn x.MetricType\n\t}\n\treturn \"\"\n}\n\ntype ExecutePlanResponse_ExecutionProgress_StageInfo struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tStageId           int64 `protobuf:\"varint,1,opt,name=stage_id,json=stageId,proto3\" json:\"stage_id,omitempty\"`\n\tNumTasks          int64 `protobuf:\"varint,2,opt,name=num_tasks,json=numTasks,proto3\" json:\"num_tasks,omitempty\"`\n\tNumCompletedTasks int64 `protobuf:\"varint,3,opt,name=num_completed_tasks,json=numCompletedTasks,proto3\" json:\"num_completed_tasks,omitempty\"`\n\tInputBytesRead    int64 `protobuf:\"varint,4,opt,name=input_bytes_read,json=inputBytesRead,proto3\" json:\"input_bytes_read,omitempty\"`\n\tDone              bool  `protobuf:\"varint,5,opt,name=done,proto3\" json:\"done,omitempty\"`\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) Reset() {\n\t*x = ExecutePlanResponse_ExecutionProgress_StageInfo{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[62]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutePlanResponse_ExecutionProgress_StageInfo) ProtoMessage() {}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[62]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutePlanResponse_ExecutionProgress_StageInfo.ProtoReflect.Descriptor instead.\nfunc (*ExecutePlanResponse_ExecutionProgress_StageInfo) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{5, 5, 0}\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetStageId() int64 {\n\tif x != nil {\n\t\treturn x.StageId\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetNumTasks() int64 {\n\tif x != nil {\n\t\treturn x.NumTasks\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetNumCompletedTasks() int64 {\n\tif x != nil {\n\t\treturn x.NumCompletedTasks\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetInputBytesRead() int64 {\n\tif x != nil {\n\t\treturn x.InputBytesRead\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetDone() bool {\n\tif x != nil {\n\t\treturn x.Done\n\t}\n\treturn false\n}\n\ntype ConfigRequest_Operation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to OpType:\n\t//\n\t//\t*ConfigRequest_Operation_Set\n\t//\t*ConfigRequest_Operation_Get\n\t//\t*ConfigRequest_Operation_GetWithDefault\n\t//\t*ConfigRequest_Operation_GetOption\n\t//\t*ConfigRequest_Operation_GetAll\n\t//\t*ConfigRequest_Operation_Unset\n\t//\t*ConfigRequest_Operation_IsModifiable\n\tOpType isConfigRequest_Operation_OpType `protobuf_oneof:\"op_type\"`\n}\n\nfunc (x *ConfigRequest_Operation) Reset() {\n\t*x = ConfigRequest_Operation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[63]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_Operation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_Operation) ProtoMessage() {}\n\nfunc (x *ConfigRequest_Operation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[63]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_Operation.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_Operation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 0}\n}\n\nfunc (m *ConfigRequest_Operation) GetOpType() isConfigRequest_Operation_OpType {\n\tif m != nil {\n\t\treturn m.OpType\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetSet() *ConfigRequest_Set {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_Set); ok {\n\t\treturn x.Set\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetGet() *ConfigRequest_Get {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_Get); ok {\n\t\treturn x.Get\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetGetWithDefault() *ConfigRequest_GetWithDefault {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_GetWithDefault); ok {\n\t\treturn x.GetWithDefault\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetGetOption() *ConfigRequest_GetOption {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_GetOption); ok {\n\t\treturn x.GetOption\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetGetAll() *ConfigRequest_GetAll {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_GetAll); ok {\n\t\treturn x.GetAll\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetUnset() *ConfigRequest_Unset {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_Unset); ok {\n\t\treturn x.Unset\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Operation) GetIsModifiable() *ConfigRequest_IsModifiable {\n\tif x, ok := x.GetOpType().(*ConfigRequest_Operation_IsModifiable); ok {\n\t\treturn x.IsModifiable\n\t}\n\treturn nil\n}\n\ntype isConfigRequest_Operation_OpType interface {\n\tisConfigRequest_Operation_OpType()\n}\n\ntype ConfigRequest_Operation_Set struct {\n\tSet *ConfigRequest_Set `protobuf:\"bytes,1,opt,name=set,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_Get struct {\n\tGet *ConfigRequest_Get `protobuf:\"bytes,2,opt,name=get,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_GetWithDefault struct {\n\tGetWithDefault *ConfigRequest_GetWithDefault `protobuf:\"bytes,3,opt,name=get_with_default,json=getWithDefault,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_GetOption struct {\n\tGetOption *ConfigRequest_GetOption `protobuf:\"bytes,4,opt,name=get_option,json=getOption,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_GetAll struct {\n\tGetAll *ConfigRequest_GetAll `protobuf:\"bytes,5,opt,name=get_all,json=getAll,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_Unset struct {\n\tUnset *ConfigRequest_Unset `protobuf:\"bytes,6,opt,name=unset,proto3,oneof\"`\n}\n\ntype ConfigRequest_Operation_IsModifiable struct {\n\tIsModifiable *ConfigRequest_IsModifiable `protobuf:\"bytes,7,opt,name=is_modifiable,json=isModifiable,proto3,oneof\"`\n}\n\nfunc (*ConfigRequest_Operation_Set) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_Get) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_GetWithDefault) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_GetOption) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_GetAll) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_Unset) isConfigRequest_Operation_OpType() {}\n\nfunc (*ConfigRequest_Operation_IsModifiable) isConfigRequest_Operation_OpType() {}\n\ntype ConfigRequest_Set struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config key-value pairs to set.\n\tPairs []*KeyValue `protobuf:\"bytes,1,rep,name=pairs,proto3\" json:\"pairs,omitempty\"`\n\t// (Optional) Whether to ignore failures.\n\tSilent *bool `protobuf:\"varint,2,opt,name=silent,proto3,oneof\" json:\"silent,omitempty\"`\n}\n\nfunc (x *ConfigRequest_Set) Reset() {\n\t*x = ConfigRequest_Set{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[64]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_Set) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_Set) ProtoMessage() {}\n\nfunc (x *ConfigRequest_Set) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[64]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_Set.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_Set) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 1}\n}\n\nfunc (x *ConfigRequest_Set) GetPairs() []*KeyValue {\n\tif x != nil {\n\t\treturn x.Pairs\n\t}\n\treturn nil\n}\n\nfunc (x *ConfigRequest_Set) GetSilent() bool {\n\tif x != nil && x.Silent != nil {\n\t\treturn *x.Silent\n\t}\n\treturn false\n}\n\ntype ConfigRequest_Get struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config keys to get.\n\tKeys []string `protobuf:\"bytes,1,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n}\n\nfunc (x *ConfigRequest_Get) Reset() {\n\t*x = ConfigRequest_Get{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[65]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_Get) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_Get) ProtoMessage() {}\n\nfunc (x *ConfigRequest_Get) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[65]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_Get.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_Get) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 2}\n}\n\nfunc (x *ConfigRequest_Get) GetKeys() []string {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\ntype ConfigRequest_GetWithDefault struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config key-value pairs to get. The value will be used as the default value.\n\tPairs []*KeyValue `protobuf:\"bytes,1,rep,name=pairs,proto3\" json:\"pairs,omitempty\"`\n}\n\nfunc (x *ConfigRequest_GetWithDefault) Reset() {\n\t*x = ConfigRequest_GetWithDefault{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[66]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_GetWithDefault) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_GetWithDefault) ProtoMessage() {}\n\nfunc (x *ConfigRequest_GetWithDefault) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[66]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_GetWithDefault.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_GetWithDefault) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 3}\n}\n\nfunc (x *ConfigRequest_GetWithDefault) GetPairs() []*KeyValue {\n\tif x != nil {\n\t\treturn x.Pairs\n\t}\n\treturn nil\n}\n\ntype ConfigRequest_GetOption struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config keys to get optionally.\n\tKeys []string `protobuf:\"bytes,1,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n}\n\nfunc (x *ConfigRequest_GetOption) Reset() {\n\t*x = ConfigRequest_GetOption{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[67]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_GetOption) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_GetOption) ProtoMessage() {}\n\nfunc (x *ConfigRequest_GetOption) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[67]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_GetOption.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_GetOption) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 4}\n}\n\nfunc (x *ConfigRequest_GetOption) GetKeys() []string {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\ntype ConfigRequest_GetAll struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The prefix of the config key to get.\n\tPrefix *string `protobuf:\"bytes,1,opt,name=prefix,proto3,oneof\" json:\"prefix,omitempty\"`\n}\n\nfunc (x *ConfigRequest_GetAll) Reset() {\n\t*x = ConfigRequest_GetAll{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[68]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_GetAll) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_GetAll) ProtoMessage() {}\n\nfunc (x *ConfigRequest_GetAll) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[68]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_GetAll.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_GetAll) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 5}\n}\n\nfunc (x *ConfigRequest_GetAll) GetPrefix() string {\n\tif x != nil && x.Prefix != nil {\n\t\treturn *x.Prefix\n\t}\n\treturn \"\"\n}\n\ntype ConfigRequest_Unset struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config keys to unset.\n\tKeys []string `protobuf:\"bytes,1,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n}\n\nfunc (x *ConfigRequest_Unset) Reset() {\n\t*x = ConfigRequest_Unset{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[69]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_Unset) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_Unset) ProtoMessage() {}\n\nfunc (x *ConfigRequest_Unset) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[69]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_Unset.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_Unset) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 6}\n}\n\nfunc (x *ConfigRequest_Unset) GetKeys() []string {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\ntype ConfigRequest_IsModifiable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The config keys to check the config is modifiable.\n\tKeys []string `protobuf:\"bytes,1,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n}\n\nfunc (x *ConfigRequest_IsModifiable) Reset() {\n\t*x = ConfigRequest_IsModifiable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[70]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ConfigRequest_IsModifiable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ConfigRequest_IsModifiable) ProtoMessage() {}\n\nfunc (x *ConfigRequest_IsModifiable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[70]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ConfigRequest_IsModifiable.ProtoReflect.Descriptor instead.\nfunc (*ConfigRequest_IsModifiable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{7, 7}\n}\n\nfunc (x *ConfigRequest_IsModifiable) GetKeys() []string {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\n// A chunk of an Artifact.\ntype AddArtifactsRequest_ArtifactChunk struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Data chunk.\n\tData []byte `protobuf:\"bytes,1,opt,name=data,proto3\" json:\"data,omitempty\"`\n\t// CRC to allow server to verify integrity of the chunk.\n\tCrc int64 `protobuf:\"varint,2,opt,name=crc,proto3\" json:\"crc,omitempty\"`\n}\n\nfunc (x *AddArtifactsRequest_ArtifactChunk) Reset() {\n\t*x = AddArtifactsRequest_ArtifactChunk{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[71]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsRequest_ArtifactChunk) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsRequest_ArtifactChunk) ProtoMessage() {}\n\nfunc (x *AddArtifactsRequest_ArtifactChunk) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[71]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsRequest_ArtifactChunk.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsRequest_ArtifactChunk) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{9, 0}\n}\n\nfunc (x *AddArtifactsRequest_ArtifactChunk) GetData() []byte {\n\tif x != nil {\n\t\treturn x.Data\n\t}\n\treturn nil\n}\n\nfunc (x *AddArtifactsRequest_ArtifactChunk) GetCrc() int64 {\n\tif x != nil {\n\t\treturn x.Crc\n\t}\n\treturn 0\n}\n\n// An artifact that is contained in a single `ArtifactChunk`.\n// Generally, this message represents tiny artifacts such as REPL-generated class files.\ntype AddArtifactsRequest_SingleChunkArtifact struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The name of the artifact is expected in the form of a \"Relative Path\" that is made up of a\n\t// sequence of directories and the final file element.\n\t// Examples of \"Relative Path\"s: \"jars/test.jar\", \"classes/xyz.class\", \"abc.xyz\", \"a/b/X.jar\".\n\t// The server is expected to maintain the hierarchy of files as defined by their name. (i.e\n\t// The relative path of the file on the server's filesystem will be the same as the name of\n\t// the provided artifact)\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// A single data chunk.\n\tData *AddArtifactsRequest_ArtifactChunk `protobuf:\"bytes,2,opt,name=data,proto3\" json:\"data,omitempty\"`\n}\n\nfunc (x *AddArtifactsRequest_SingleChunkArtifact) Reset() {\n\t*x = AddArtifactsRequest_SingleChunkArtifact{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[72]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsRequest_SingleChunkArtifact) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsRequest_SingleChunkArtifact) ProtoMessage() {}\n\nfunc (x *AddArtifactsRequest_SingleChunkArtifact) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[72]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsRequest_SingleChunkArtifact.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsRequest_SingleChunkArtifact) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{9, 1}\n}\n\nfunc (x *AddArtifactsRequest_SingleChunkArtifact) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsRequest_SingleChunkArtifact) GetData() *AddArtifactsRequest_ArtifactChunk {\n\tif x != nil {\n\t\treturn x.Data\n\t}\n\treturn nil\n}\n\n// A number of `SingleChunkArtifact` batched into a single RPC.\ntype AddArtifactsRequest_Batch struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tArtifacts []*AddArtifactsRequest_SingleChunkArtifact `protobuf:\"bytes,1,rep,name=artifacts,proto3\" json:\"artifacts,omitempty\"`\n}\n\nfunc (x *AddArtifactsRequest_Batch) Reset() {\n\t*x = AddArtifactsRequest_Batch{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[73]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsRequest_Batch) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsRequest_Batch) ProtoMessage() {}\n\nfunc (x *AddArtifactsRequest_Batch) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[73]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsRequest_Batch.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsRequest_Batch) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{9, 2}\n}\n\nfunc (x *AddArtifactsRequest_Batch) GetArtifacts() []*AddArtifactsRequest_SingleChunkArtifact {\n\tif x != nil {\n\t\treturn x.Artifacts\n\t}\n\treturn nil\n}\n\n// Signals the beginning/start of a chunked artifact.\n// A large artifact is transferred through a payload of `BeginChunkedArtifact` followed by a\n// sequence of `ArtifactChunk`s.\ntype AddArtifactsRequest_BeginChunkedArtifact struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Name of the artifact undergoing chunking. Follows the same conventions as the `name` in\n\t// the `Artifact` message.\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// Total size of the artifact in bytes.\n\tTotalBytes int64 `protobuf:\"varint,2,opt,name=total_bytes,json=totalBytes,proto3\" json:\"total_bytes,omitempty\"`\n\t// Number of chunks the artifact is split into.\n\t// This includes the `initial_chunk`.\n\tNumChunks int64 `protobuf:\"varint,3,opt,name=num_chunks,json=numChunks,proto3\" json:\"num_chunks,omitempty\"`\n\t// The first/initial chunk.\n\tInitialChunk *AddArtifactsRequest_ArtifactChunk `protobuf:\"bytes,4,opt,name=initial_chunk,json=initialChunk,proto3\" json:\"initial_chunk,omitempty\"`\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) Reset() {\n\t*x = AddArtifactsRequest_BeginChunkedArtifact{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[74]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsRequest_BeginChunkedArtifact) ProtoMessage() {}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[74]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsRequest_BeginChunkedArtifact.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsRequest_BeginChunkedArtifact) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{9, 3}\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) GetTotalBytes() int64 {\n\tif x != nil {\n\t\treturn x.TotalBytes\n\t}\n\treturn 0\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) GetNumChunks() int64 {\n\tif x != nil {\n\t\treturn x.NumChunks\n\t}\n\treturn 0\n}\n\nfunc (x *AddArtifactsRequest_BeginChunkedArtifact) GetInitialChunk() *AddArtifactsRequest_ArtifactChunk {\n\tif x != nil {\n\t\treturn x.InitialChunk\n\t}\n\treturn nil\n}\n\n// Metadata of an artifact.\ntype AddArtifactsResponse_ArtifactSummary struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// Whether the CRC (Cyclic Redundancy Check) is successful on server verification.\n\t// The server discards any artifact that fails the CRC.\n\t// If false, the client may choose to resend the artifact specified by `name`.\n\tIsCrcSuccessful bool `protobuf:\"varint,2,opt,name=is_crc_successful,json=isCrcSuccessful,proto3\" json:\"is_crc_successful,omitempty\"`\n}\n\nfunc (x *AddArtifactsResponse_ArtifactSummary) Reset() {\n\t*x = AddArtifactsResponse_ArtifactSummary{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[75]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AddArtifactsResponse_ArtifactSummary) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AddArtifactsResponse_ArtifactSummary) ProtoMessage() {}\n\nfunc (x *AddArtifactsResponse_ArtifactSummary) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[75]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AddArtifactsResponse_ArtifactSummary.ProtoReflect.Descriptor instead.\nfunc (*AddArtifactsResponse_ArtifactSummary) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{10, 0}\n}\n\nfunc (x *AddArtifactsResponse_ArtifactSummary) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *AddArtifactsResponse_ArtifactSummary) GetIsCrcSuccessful() bool {\n\tif x != nil {\n\t\treturn x.IsCrcSuccessful\n\t}\n\treturn false\n}\n\ntype ArtifactStatusesResponse_ArtifactStatus struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Exists or not particular artifact at the server.\n\tExists bool `protobuf:\"varint,1,opt,name=exists,proto3\" json:\"exists,omitempty\"`\n}\n\nfunc (x *ArtifactStatusesResponse_ArtifactStatus) Reset() {\n\t*x = ArtifactStatusesResponse_ArtifactStatus{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[77]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ArtifactStatusesResponse_ArtifactStatus) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ArtifactStatusesResponse_ArtifactStatus) ProtoMessage() {}\n\nfunc (x *ArtifactStatusesResponse_ArtifactStatus) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[77]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ArtifactStatusesResponse_ArtifactStatus.ProtoReflect.Descriptor instead.\nfunc (*ArtifactStatusesResponse_ArtifactStatus) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{12, 1}\n}\n\nfunc (x *ArtifactStatusesResponse_ArtifactStatus) GetExists() bool {\n\tif x != nil {\n\t\treturn x.Exists\n\t}\n\treturn false\n}\n\n// Release and close operation completely.\n// This will also interrupt the query if it is running execution, and wait for it to be torn down.\ntype ReleaseExecuteRequest_ReleaseAll struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *ReleaseExecuteRequest_ReleaseAll) Reset() {\n\t*x = ReleaseExecuteRequest_ReleaseAll{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[78]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseExecuteRequest_ReleaseAll) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseExecuteRequest_ReleaseAll) ProtoMessage() {}\n\nfunc (x *ReleaseExecuteRequest_ReleaseAll) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[78]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseExecuteRequest_ReleaseAll.ProtoReflect.Descriptor instead.\nfunc (*ReleaseExecuteRequest_ReleaseAll) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{17, 0}\n}\n\n// Release all responses from the operation response stream up to and including\n// the response with the given by response_id.\n// While server determines by itself how much of a buffer of responses to keep, client providing\n// explicit release calls will help reduce resource consumption.\n// Noop if response_id not found in cached responses.\ntype ReleaseExecuteRequest_ReleaseUntil struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tResponseId string `protobuf:\"bytes,1,opt,name=response_id,json=responseId,proto3\" json:\"response_id,omitempty\"`\n}\n\nfunc (x *ReleaseExecuteRequest_ReleaseUntil) Reset() {\n\t*x = ReleaseExecuteRequest_ReleaseUntil{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[79]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ReleaseExecuteRequest_ReleaseUntil) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ReleaseExecuteRequest_ReleaseUntil) ProtoMessage() {}\n\nfunc (x *ReleaseExecuteRequest_ReleaseUntil) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[79]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ReleaseExecuteRequest_ReleaseUntil.ProtoReflect.Descriptor instead.\nfunc (*ReleaseExecuteRequest_ReleaseUntil) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{17, 1}\n}\n\nfunc (x *ReleaseExecuteRequest_ReleaseUntil) GetResponseId() string {\n\tif x != nil {\n\t\treturn x.ResponseId\n\t}\n\treturn \"\"\n}\n\ntype FetchErrorDetailsResponse_StackTraceElement struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The fully qualified name of the class containing the execution point.\n\tDeclaringClass string `protobuf:\"bytes,1,opt,name=declaring_class,json=declaringClass,proto3\" json:\"declaring_class,omitempty\"`\n\t// The name of the method containing the execution point.\n\tMethodName string `protobuf:\"bytes,2,opt,name=method_name,json=methodName,proto3\" json:\"method_name,omitempty\"`\n\t// The name of the file containing the execution point.\n\tFileName *string `protobuf:\"bytes,3,opt,name=file_name,json=fileName,proto3,oneof\" json:\"file_name,omitempty\"`\n\t// The line number of the source line containing the execution point.\n\tLineNumber int32 `protobuf:\"varint,4,opt,name=line_number,json=lineNumber,proto3\" json:\"line_number,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) Reset() {\n\t*x = FetchErrorDetailsResponse_StackTraceElement{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[80]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsResponse_StackTraceElement) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[80]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse_StackTraceElement.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsResponse_StackTraceElement) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22, 0}\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) GetDeclaringClass() string {\n\tif x != nil {\n\t\treturn x.DeclaringClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) GetMethodName() string {\n\tif x != nil {\n\t\treturn x.MethodName\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) GetFileName() string {\n\tif x != nil && x.FileName != nil {\n\t\treturn *x.FileName\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_StackTraceElement) GetLineNumber() int32 {\n\tif x != nil {\n\t\treturn x.LineNumber\n\t}\n\treturn 0\n}\n\n// QueryContext defines the schema for the query context of a SparkThrowable.\n// It helps users understand where the error occurs while executing queries.\ntype FetchErrorDetailsResponse_QueryContext struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tContextType FetchErrorDetailsResponse_QueryContext_ContextType `protobuf:\"varint,10,opt,name=context_type,json=contextType,proto3,enum=spark.connect.FetchErrorDetailsResponse_QueryContext_ContextType\" json:\"context_type,omitempty\"`\n\t// The object type of the query which throws the exception.\n\t// If the exception is directly from the main query, it should be an empty string.\n\t// Otherwise, it should be the exact object type in upper case. For example, a \"VIEW\".\n\tObjectType string `protobuf:\"bytes,1,opt,name=object_type,json=objectType,proto3\" json:\"object_type,omitempty\"`\n\t// The object name of the query which throws the exception.\n\t// If the exception is directly from the main query, it should be an empty string.\n\t// Otherwise, it should be the object name. For example, a view name \"V1\".\n\tObjectName string `protobuf:\"bytes,2,opt,name=object_name,json=objectName,proto3\" json:\"object_name,omitempty\"`\n\t// The starting index in the query text which throws the exception. The index starts from 0.\n\tStartIndex int32 `protobuf:\"varint,3,opt,name=start_index,json=startIndex,proto3\" json:\"start_index,omitempty\"`\n\t// The stopping index in the query which throws the exception. The index starts from 0.\n\tStopIndex int32 `protobuf:\"varint,4,opt,name=stop_index,json=stopIndex,proto3\" json:\"stop_index,omitempty\"`\n\t// The corresponding fragment of the query which throws the exception.\n\tFragment string `protobuf:\"bytes,5,opt,name=fragment,proto3\" json:\"fragment,omitempty\"`\n\t// The user code (call site of the API) that caused throwing the exception.\n\tCallSite string `protobuf:\"bytes,6,opt,name=call_site,json=callSite,proto3\" json:\"call_site,omitempty\"`\n\t// Summary of the exception cause.\n\tSummary string `protobuf:\"bytes,7,opt,name=summary,proto3\" json:\"summary,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) Reset() {\n\t*x = FetchErrorDetailsResponse_QueryContext{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[81]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsResponse_QueryContext) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[81]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse_QueryContext.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsResponse_QueryContext) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22, 1}\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetContextType() FetchErrorDetailsResponse_QueryContext_ContextType {\n\tif x != nil {\n\t\treturn x.ContextType\n\t}\n\treturn FetchErrorDetailsResponse_QueryContext_SQL\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetObjectType() string {\n\tif x != nil {\n\t\treturn x.ObjectType\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetObjectName() string {\n\tif x != nil {\n\t\treturn x.ObjectName\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetStartIndex() int32 {\n\tif x != nil {\n\t\treturn x.StartIndex\n\t}\n\treturn 0\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetStopIndex() int32 {\n\tif x != nil {\n\t\treturn x.StopIndex\n\t}\n\treturn 0\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetFragment() string {\n\tif x != nil {\n\t\treturn x.Fragment\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetCallSite() string {\n\tif x != nil {\n\t\treturn x.CallSite\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_QueryContext) GetSummary() string {\n\tif x != nil {\n\t\treturn x.Summary\n\t}\n\treturn \"\"\n}\n\n// SparkThrowable defines the schema for SparkThrowable exceptions.\ntype FetchErrorDetailsResponse_SparkThrowable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Succinct, human-readable, unique, and consistent representation of the error category.\n\tErrorClass *string `protobuf:\"bytes,1,opt,name=error_class,json=errorClass,proto3,oneof\" json:\"error_class,omitempty\"`\n\t// The message parameters for the error framework.\n\tMessageParameters map[string]string `protobuf:\"bytes,2,rep,name=message_parameters,json=messageParameters,proto3\" json:\"message_parameters,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// The query context of a SparkThrowable.\n\tQueryContexts []*FetchErrorDetailsResponse_QueryContext `protobuf:\"bytes,3,rep,name=query_contexts,json=queryContexts,proto3\" json:\"query_contexts,omitempty\"`\n\t// Portable error identifier across SQL engines\n\t// If null, error class or SQLSTATE is not set.\n\tSqlState *string `protobuf:\"bytes,4,opt,name=sql_state,json=sqlState,proto3,oneof\" json:\"sql_state,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) Reset() {\n\t*x = FetchErrorDetailsResponse_SparkThrowable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[82]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsResponse_SparkThrowable) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[82]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse_SparkThrowable.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsResponse_SparkThrowable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22, 2}\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) GetErrorClass() string {\n\tif x != nil && x.ErrorClass != nil {\n\t\treturn *x.ErrorClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) GetMessageParameters() map[string]string {\n\tif x != nil {\n\t\treturn x.MessageParameters\n\t}\n\treturn nil\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) GetQueryContexts() []*FetchErrorDetailsResponse_QueryContext {\n\tif x != nil {\n\t\treturn x.QueryContexts\n\t}\n\treturn nil\n}\n\nfunc (x *FetchErrorDetailsResponse_SparkThrowable) GetSqlState() string {\n\tif x != nil && x.SqlState != nil {\n\t\treturn *x.SqlState\n\t}\n\treturn \"\"\n}\n\n// Error defines the schema for the representing exception.\ntype FetchErrorDetailsResponse_Error struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The fully qualified names of the exception class and its parent classes.\n\tErrorTypeHierarchy []string `protobuf:\"bytes,1,rep,name=error_type_hierarchy,json=errorTypeHierarchy,proto3\" json:\"error_type_hierarchy,omitempty\"`\n\t// The detailed message of the exception.\n\tMessage string `protobuf:\"bytes,2,opt,name=message,proto3\" json:\"message,omitempty\"`\n\t// The stackTrace of the exception. It will be set\n\t// if the SQLConf spark.sql.connect.serverStacktrace.enabled is true.\n\tStackTrace []*FetchErrorDetailsResponse_StackTraceElement `protobuf:\"bytes,3,rep,name=stack_trace,json=stackTrace,proto3\" json:\"stack_trace,omitempty\"`\n\t// The index of the cause error in errors.\n\tCauseIdx *int32 `protobuf:\"varint,4,opt,name=cause_idx,json=causeIdx,proto3,oneof\" json:\"cause_idx,omitempty\"`\n\t// The structured data of a SparkThrowable exception.\n\tSparkThrowable *FetchErrorDetailsResponse_SparkThrowable `protobuf:\"bytes,5,opt,name=spark_throwable,json=sparkThrowable,proto3,oneof\" json:\"spark_throwable,omitempty\"`\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) Reset() {\n\t*x = FetchErrorDetailsResponse_Error{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_base_proto_msgTypes[83]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FetchErrorDetailsResponse_Error) ProtoMessage() {}\n\nfunc (x *FetchErrorDetailsResponse_Error) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_base_proto_msgTypes[83]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FetchErrorDetailsResponse_Error.ProtoReflect.Descriptor instead.\nfunc (*FetchErrorDetailsResponse_Error) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_base_proto_rawDescGZIP(), []int{22, 3}\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) GetErrorTypeHierarchy() []string {\n\tif x != nil {\n\t\treturn x.ErrorTypeHierarchy\n\t}\n\treturn nil\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) GetMessage() string {\n\tif x != nil {\n\t\treturn x.Message\n\t}\n\treturn \"\"\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) GetStackTrace() []*FetchErrorDetailsResponse_StackTraceElement {\n\tif x != nil {\n\t\treturn x.StackTrace\n\t}\n\treturn nil\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) GetCauseIdx() int32 {\n\tif x != nil && x.CauseIdx != nil {\n\t\treturn *x.CauseIdx\n\t}\n\treturn 0\n}\n\nfunc (x *FetchErrorDetailsResponse_Error) GetSparkThrowable() *FetchErrorDetailsResponse_SparkThrowable {\n\tif x != nil {\n\t\treturn x.SparkThrowable\n\t}\n\treturn nil\n}\n\nvar File_spark_connect_base_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_base_proto_rawDesc = []byte{\n\t0x0a, 0x18, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x62, 0x61, 0x73, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c,\n\t0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70,\n\t0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x2e, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x1a, 0x1a, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x65, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a,\n\t0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x72,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x74, 0x79,\n\t0x70, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x16, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x6d, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74,\n\t0x6f, 0x1a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,\n\t0x22, 0x74, 0x0a, 0x04, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x2d, 0x0a, 0x04, 0x72, 0x6f, 0x6f, 0x74,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48,\n\t0x00, 0x52, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x12, 0x32, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x48, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x6f,\n\t0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x7a, 0x0a, 0x0b, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f,\n\t0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x69, 0x64,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x75, 0x73, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b,\n\t0x0a, 0x09, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x08, 0x75, 0x73, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x35, 0x0a, 0x0a, 0x65,\n\t0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xe7, 0x07, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,\n\t0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x0a, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f,\n\t0x6e, 0x73, 0x22, 0xf5, 0x14, 0x0a, 0x12, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c,\n\t0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65,\n\t0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65,\n\t0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72,\n\t0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01,\n\t0x12, 0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65,\n\t0x78, 0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12,\n\t0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79,\n\t0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x42, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18,\n\t0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61,\n\t0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x48,\n\t0x00, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x45, 0x0a, 0x07, 0x65, 0x78, 0x70,\n\t0x6c, 0x61, 0x69, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79,\n\t0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x6c, 0x61, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e,\n\t0x12, 0x4f, 0x0a, 0x0b, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18,\n\t0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61,\n\t0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x54, 0x72, 0x65, 0x65, 0x53, 0x74, 0x72,\n\t0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x72, 0x65, 0x65, 0x53, 0x74, 0x72, 0x69, 0x6e,\n\t0x67, 0x12, 0x46, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x18, 0x07, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x48, 0x00,\n\t0x52, 0x07, 0x69, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x52, 0x0a, 0x0c, 0x69, 0x73, 0x5f,\n\t0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65,\n\t0x73, 0x74, 0x2e, 0x49, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x48, 0x00,\n\t0x52, 0x0b, 0x69, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x12, 0x4f, 0x0a,\n\t0x0b, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65,\n\t0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73,\n\t0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x55,\n\t0x0a, 0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18,\n\t0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61,\n\t0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65,\n\t0x72, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65,\n\t0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x49, 0x0a, 0x09, 0x64, 0x64, 0x6c, 0x5f, 0x70, 0x61, 0x72,\n\t0x73, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65,\n\t0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x44, 0x4c, 0x50,\n\t0x61, 0x72, 0x73, 0x65, 0x48, 0x00, 0x52, 0x08, 0x64, 0x64, 0x6c, 0x50, 0x61, 0x72, 0x73, 0x65,\n\t0x12, 0x58, 0x0a, 0x0e, 0x73, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69,\n\t0x63, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65,\n\t0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x61, 0x6d, 0x65,\n\t0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x73, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x61, 0x6d,\n\t0x65, 0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x73, 0x12, 0x55, 0x0a, 0x0d, 0x73, 0x65,\n\t0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71,\n\t0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73,\n\t0x68, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73,\n\t0x68, 0x12, 0x45, 0x0a, 0x07, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x18, 0x0e, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65,\n\t0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x48, 0x00, 0x52,\n\t0x07, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x12, 0x4b, 0x0a, 0x09, 0x75, 0x6e, 0x70, 0x65,\n\t0x72, 0x73, 0x69, 0x73, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c,\n\t0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x55,\n\t0x6e, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x75, 0x6e, 0x70, 0x65,\n\t0x72, 0x73, 0x69, 0x73, 0x74, 0x12, 0x5f, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x6f,\n\t0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75,\n\t0x65, 0x73, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65,\n\t0x76, 0x65, 0x6c, 0x48, 0x00, 0x52, 0x0f, 0x67, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67,\n\t0x65, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x4d, 0x0a, 0x0b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x74,\n\t0x6f, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c,\n\t0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4a,\n\t0x73, 0x6f, 0x6e, 0x54, 0x6f, 0x44, 0x44, 0x4c, 0x48, 0x00, 0x52, 0x09, 0x6a, 0x73, 0x6f, 0x6e,\n\t0x54, 0x6f, 0x44, 0x64, 0x6c, 0x1a, 0x31, 0x0a, 0x06, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12,\n\t0x27, 0x0a, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c,\n\t0x61, 0x6e, 0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x1a, 0xbb, 0x02, 0x0a, 0x07, 0x45, 0x78, 0x70,\n\t0x6c, 0x61, 0x69, 0x6e, 0x12, 0x27, 0x0a, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x12, 0x58, 0x0a,\n\t0x0c, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0e, 0x32, 0x35, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x2e, 0x45,\n\t0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0b, 0x65, 0x78, 0x70, 0x6c,\n\t0x61, 0x69, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0xac, 0x01, 0x0a, 0x0b, 0x45, 0x78, 0x70, 0x6c,\n\t0x61, 0x69, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x18, 0x45, 0x58, 0x50, 0x4c, 0x41,\n\t0x49, 0x4e, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46,\n\t0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x45, 0x58, 0x50, 0x4c, 0x41, 0x49, 0x4e,\n\t0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x53, 0x49, 0x4d, 0x50, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x19,\n\t0x0a, 0x15, 0x45, 0x58, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x45,\n\t0x58, 0x54, 0x45, 0x4e, 0x44, 0x45, 0x44, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x45, 0x58, 0x50,\n\t0x4c, 0x41, 0x49, 0x4e, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x47, 0x45,\n\t0x4e, 0x10, 0x03, 0x12, 0x15, 0x0a, 0x11, 0x45, 0x58, 0x50, 0x4c, 0x41, 0x49, 0x4e, 0x5f, 0x4d,\n\t0x4f, 0x44, 0x45, 0x5f, 0x43, 0x4f, 0x53, 0x54, 0x10, 0x04, 0x12, 0x1a, 0x0a, 0x16, 0x45, 0x58,\n\t0x50, 0x4c, 0x41, 0x49, 0x4e, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x4d, 0x41,\n\t0x54, 0x54, 0x45, 0x44, 0x10, 0x05, 0x1a, 0x5a, 0x0a, 0x0a, 0x54, 0x72, 0x65, 0x65, 0x53, 0x74,\n\t0x72, 0x69, 0x6e, 0x67, 0x12, 0x27, 0x0a, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x12, 0x19, 0x0a,\n\t0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x05,\n\t0x6c, 0x65, 0x76, 0x65, 0x6c, 0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x6c, 0x65, 0x76,\n\t0x65, 0x6c, 0x1a, 0x32, 0x0a, 0x07, 0x49, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x27, 0x0a,\n\t0x04, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e,\n\t0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x1a, 0x36, 0x0a, 0x0b, 0x49, 0x73, 0x53, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x12, 0x27, 0x0a, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x1a, 0x35,\n\t0x0a, 0x0a, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x27, 0x0a, 0x04,\n\t0x70, 0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x04, 0x70, 0x6c, 0x61, 0x6e, 0x1a, 0x0e, 0x0a, 0x0c, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65,\n\t0x72, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0x29, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x50, 0x61, 0x72, 0x73,\n\t0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x64, 0x64, 0x6c, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67,\n\t0x1a, 0x79, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63,\n\t0x73, 0x12, 0x34, 0x0a, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x70, 0x6c, 0x61, 0x6e,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x0a, 0x74, 0x61, 0x72,\n\t0x67, 0x65, 0x74, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x32, 0x0a, 0x0a, 0x6f, 0x74, 0x68, 0x65, 0x72,\n\t0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e,\n\t0x52, 0x09, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x50, 0x6c, 0x61, 0x6e, 0x1a, 0x37, 0x0a, 0x0c, 0x53,\n\t0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73, 0x68, 0x12, 0x27, 0x0a, 0x04, 0x70,\n\t0x6c, 0x61, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x04,\n\t0x70, 0x6c, 0x61, 0x6e, 0x1a, 0x97, 0x01, 0x0a, 0x07, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74,\n\t0x12, 0x33, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65,\n\t0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x6f,\n\t0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74, 0x6f,\n\t0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x88, 0x01, 0x01, 0x42, 0x10, 0x0a, 0x0e,\n\t0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x1a, 0x6e,\n\t0x0a, 0x09, 0x55, 0x6e, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x12, 0x33, 0x0a, 0x08, 0x72,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x12, 0x1f, 0x0a, 0x08, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x08, 0x48, 0x00, 0x52, 0x08, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x69, 0x6e, 0x67, 0x88, 0x01,\n\t0x01, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x69, 0x6e, 0x67, 0x1a, 0x46,\n\t0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65,\n\t0x6c, 0x12, 0x33, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x2c, 0x0a, 0x09, 0x4a, 0x73, 0x6f, 0x6e, 0x54, 0x6f,\n\t0x44, 0x44, 0x4c, 0x12, 0x1f, 0x0a, 0x0b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x72, 0x69,\n\t0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74,\n\t0x72, 0x69, 0x6e, 0x67, 0x42, 0x09, 0x0a, 0x07, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x42,\n\t0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72,\n\t0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f,\n\t0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xca, 0x0e, 0x0a, 0x13, 0x41,\n\t0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,\n\t0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49,\n\t0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65,\n\t0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x0f, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x43, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c,\n\t0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d,\n\t0x61, 0x48, 0x00, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x46, 0x0a, 0x07, 0x65,\n\t0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61,\n\t0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,\n\t0x2e, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x65, 0x78, 0x70, 0x6c,\n\t0x61, 0x69, 0x6e, 0x12, 0x50, 0x0a, 0x0b, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x73, 0x74, 0x72, 0x69,\n\t0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65,\n\t0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x54, 0x72, 0x65,\n\t0x65, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x72, 0x65, 0x65, 0x53,\n\t0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x47, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x6c, 0x6f, 0x63, 0x61,\n\t0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50,\n\t0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x49, 0x73, 0x4c, 0x6f,\n\t0x63, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x07, 0x69, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x53,\n\t0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x06,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e,\n\t0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x49, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61,\n\t0x6d, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x0b, 0x69, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,\n\t0x69, 0x6e, 0x67, 0x12, 0x50, 0x0a, 0x0b, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x6c,\n\t0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65,\n\t0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x49, 0x6e, 0x70,\n\t0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x56, 0x0a, 0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x5f, 0x76,\n\t0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61,\n\t0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,\n\t0x2e, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52,\n\t0x0c, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a,\n\t0x09, 0x64, 0x64, 0x6c, 0x5f, 0x70, 0x61, 0x72, 0x73, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x44, 0x44, 0x4c, 0x50, 0x61, 0x72, 0x73, 0x65, 0x48, 0x00, 0x52,\n\t0x08, 0x64, 0x64, 0x6c, 0x50, 0x61, 0x72, 0x73, 0x65, 0x12, 0x59, 0x0a, 0x0e, 0x73, 0x61, 0x6d,\n\t0x65, 0x5f, 0x73, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x61, 0x6d, 0x65, 0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74,\n\t0x69, 0x63, 0x73, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x61, 0x6d, 0x65, 0x53, 0x65, 0x6d, 0x61, 0x6e,\n\t0x74, 0x69, 0x63, 0x73, 0x12, 0x56, 0x0a, 0x0d, 0x73, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63,\n\t0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c,\n\t0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e,\n\t0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73, 0x68, 0x48, 0x00, 0x52, 0x0c,\n\t0x73, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73, 0x68, 0x12, 0x46, 0x0a, 0x07,\n\t0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e,\n\t0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x2e, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x65, 0x72,\n\t0x73, 0x69, 0x73, 0x74, 0x12, 0x4c, 0x0a, 0x09, 0x75, 0x6e, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73,\n\t0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50,\n\t0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x55, 0x6e, 0x70, 0x65,\n\t0x72, 0x73, 0x69, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x75, 0x6e, 0x70, 0x65, 0x72, 0x73, 0x69,\n\t0x73, 0x74, 0x12, 0x60, 0x0a, 0x11, 0x67, 0x65, 0x74, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67,\n\t0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e,\n\t0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65,\n\t0x6c, 0x48, 0x00, 0x52, 0x0f, 0x67, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c,\n\t0x65, 0x76, 0x65, 0x6c, 0x12, 0x4e, 0x0a, 0x0b, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x5f,\n\t0x64, 0x64, 0x6c, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a,\n\t0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4a, 0x73,\n\t0x6f, 0x6e, 0x54, 0x6f, 0x44, 0x44, 0x4c, 0x48, 0x00, 0x52, 0x09, 0x6a, 0x73, 0x6f, 0x6e, 0x54,\n\t0x6f, 0x44, 0x64, 0x6c, 0x1a, 0x39, 0x0a, 0x06, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x2f,\n\t0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x1a,\n\t0x30, 0x0a, 0x07, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78,\n\t0x70, 0x6c, 0x61, 0x69, 0x6e, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x0d, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x53, 0x74, 0x72, 0x69, 0x6e,\n\t0x67, 0x1a, 0x2d, 0x0a, 0x0a, 0x54, 0x72, 0x65, 0x65, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12,\n\t0x1f, 0x0a, 0x0b, 0x74, 0x72, 0x65, 0x65, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x72, 0x65, 0x65, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67,\n\t0x1a, 0x24, 0x0a, 0x07, 0x49, 0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x19, 0x0a, 0x08, 0x69,\n\t0x73, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69,\n\t0x73, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x1a, 0x30, 0x0a, 0x0b, 0x49, 0x73, 0x53, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x53,\n\t0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x1a, 0x22, 0x0a, 0x0a, 0x49, 0x6e, 0x70, 0x75,\n\t0x74, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18,\n\t0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x1a, 0x28, 0x0a, 0x0c,\n\t0x53, 0x70, 0x61, 0x72, 0x6b, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07,\n\t0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76,\n\t0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0x3b, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x50, 0x61, 0x72,\n\t0x73, 0x65, 0x12, 0x2f, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, 0x70, 0x61, 0x72,\n\t0x73, 0x65, 0x64, 0x1a, 0x27, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x53, 0x65, 0x6d, 0x61, 0x6e,\n\t0x74, 0x69, 0x63, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0x26, 0x0a, 0x0c,\n\t0x53, 0x65, 0x6d, 0x61, 0x6e, 0x74, 0x69, 0x63, 0x48, 0x61, 0x73, 0x68, 0x12, 0x16, 0x0a, 0x06,\n\t0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x72, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x1a, 0x09, 0x0a, 0x07, 0x50, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x1a,\n\t0x0b, 0x0a, 0x09, 0x55, 0x6e, 0x70, 0x65, 0x72, 0x73, 0x69, 0x73, 0x74, 0x1a, 0x53, 0x0a, 0x0f,\n\t0x47, 0x65, 0x74, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12,\n\t0x40, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65,\n\t0x76, 0x65, 0x6c, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65,\n\t0x6c, 0x1a, 0x2a, 0x0a, 0x09, 0x4a, 0x73, 0x6f, 0x6e, 0x54, 0x6f, 0x44, 0x44, 0x4c, 0x12, 0x1d,\n\t0x0a, 0x0a, 0x64, 0x64, 0x6c, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x09, 0x64, 0x64, 0x6c, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x08, 0x0a,\n\t0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0xa3, 0x05, 0x0a, 0x12, 0x45, 0x78, 0x65, 0x63,\n\t0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d,\n\t0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a,\n\t0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64,\n\t0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,\n\t0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53,\n\t0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f,\n\t0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72,\n\t0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e,\n\t0x74, 0x65, 0x78, 0x74, 0x12, 0x26, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0b, 0x6f, 0x70,\n\t0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x27, 0x0a, 0x04,\n\t0x70, 0x6c, 0x61, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x04, 0x70, 0x6c, 0x61, 0x6e, 0x12, 0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0a, 0x63, 0x6c,\n\t0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x58, 0x0a, 0x0f, 0x72,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e,\n\t0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4f,\n\t0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x07, 0x20,\n\t0x03, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x1a, 0xa5, 0x01, 0x0a, 0x0d, 0x52, 0x65,\n\t0x71, 0x75, 0x65, 0x73, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x0a, 0x10, 0x72,\n\t0x65, 0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x72, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63,\n\t0x68, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x35, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65,\n\t0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67,\n\t0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41,\n\t0x6e, 0x79, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x42,\n\t0x10, 0x0a, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73,\n\t0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64,\n\t0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0f, 0x0a, 0x0d,\n\t0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a,\n\t0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xee, 0x18,\n\t0x0a, 0x13, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73,\n\t0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x0f,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65,\n\t0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b,\n\t0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x49, 0x64, 0x12, 0x50, 0x0a,\n\t0x0b, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65,\n\t0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x72, 0x72, 0x6f, 0x77, 0x42, 0x61, 0x74, 0x63,\n\t0x68, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x72, 0x72, 0x6f, 0x77, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12,\n\t0x63, 0x0a, 0x12, 0x73, 0x71, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72,\n\t0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63,\n\t0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e,\n\t0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x48, 0x00, 0x52, 0x10, 0x73, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x12, 0x7e, 0x0a, 0x23, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x73, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73,\n\t0x74, 0x61, 0x72, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c,\n\t0x74, 0x48, 0x00, 0x52, 0x1f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,\n\t0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x12, 0x71, 0x0a, 0x1e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,\n\t0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f,\n\t0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72,\n\t0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x1b, 0x73, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x6b, 0x0a, 0x1c, 0x67, 0x65, 0x74, 0x5f, 0x72,\n\t0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x47, 0x65,\n\t0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x19, 0x67, 0x65, 0x74, 0x52, 0x65,\n\t0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x12, 0x87, 0x01, 0x0a, 0x26, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,\n\t0x6e, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72,\n\t0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18,\n\t0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x22, 0x73, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65,\n\t0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x87,\n\t0x01, 0x0a, 0x26, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x71, 0x75, 0x65,\n\t0x72, 0x79, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x65, 0x76, 0x65, 0x6e,\n\t0x74, 0x73, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69,\n\t0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x48, 0x00, 0x52, 0x22, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e,\n\t0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x5c, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x43, 0x6f, 0x6d, 0x70,\n\t0x6c, 0x65, 0x74, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x43, 0x6f,\n\t0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x87, 0x01, 0x0a, 0x26, 0x63, 0x72, 0x65, 0x61, 0x74,\n\t0x65, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x66, 0x69,\n\t0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c,\n\t0x74, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65,\n\t0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x22, 0x63, 0x72,\n\t0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x66,\n\t0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x12, 0x65, 0x0a, 0x12, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72,\n\t0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65,\n\t0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,\n\t0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65,\n\t0x73, 0x73, 0x48, 0x00, 0x52, 0x11, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50,\n\t0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x64, 0x0a, 0x19, 0x63, 0x68, 0x65, 0x63, 0x6b,\n\t0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b,\n\t0x70, 0x6f, 0x69, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x48, 0x00, 0x52, 0x17, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x4c, 0x0a,\n\t0x11, 0x6d, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x0f, 0x6d, 0x6c, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x58, 0x0a, 0x15, 0x70,\n\t0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c,\n\t0x69, 0x6e, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00,\n\t0x52, 0x13, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52,\n\t0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x5e, 0x0a, 0x17, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,\n\t0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x15,\n\t0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52,\n\t0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69,\n\t0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67,\n\t0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x48,\n\t0x00, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x44, 0x0a, 0x07,\n\t0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69,\n\t0x63, 0x73, 0x12, 0x5d, 0x0a, 0x10, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x6d,\n\t0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65,\n\t0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,\n\t0x2e, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73,\n\t0x52, 0x0f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63,\n\t0x73, 0x12, 0x2f, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65,\n\t0x6d, 0x61, 0x1a, 0x47, 0x0a, 0x10, 0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x33, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x76, 0x0a, 0x0a, 0x41,\n\t0x72, 0x72, 0x6f, 0x77, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x6f, 0x77,\n\t0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x72, 0x6f,\n\t0x77, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x26, 0x0a, 0x0c, 0x73, 0x74,\n\t0x61, 0x72, 0x74, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,\n\t0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x88,\n\t0x01, 0x01, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, 0x66, 0x66,\n\t0x73, 0x65, 0x74, 0x1a, 0x85, 0x04, 0x0a, 0x07, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12,\n\t0x51, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x4d, 0x65, 0x74,\n\t0x72, 0x69, 0x63, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69,\n\t0x63, 0x73, 0x1a, 0xcc, 0x02, 0x0a, 0x0c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x4f, 0x62, 0x6a,\n\t0x65, 0x63, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x64,\n\t0x12, 0x16, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,\n\t0x52, 0x06, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x7a, 0x0a, 0x11, 0x65, 0x78, 0x65, 0x63,\n\t0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x04, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x4d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e,\n\t0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65,\n\t0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74,\n\t0x72, 0x79, 0x52, 0x10, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74,\n\t0x72, 0x69, 0x63, 0x73, 0x1a, 0x7b, 0x0a, 0x15, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,\n\t0x6e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,\n\t0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,\n\t0x4c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,\n\t0x73, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x4d, 0x65, 0x74, 0x72, 0x69,\n\t0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,\n\t0x01, 0x1a, 0x58, 0x0a, 0x0b, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65,\n\t0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,\n\t0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x65,\n\t0x74, 0x72, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x0a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x1a, 0x8d, 0x01, 0x0a, 0x0f,\n\t0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12,\n\t0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,\n\t0x61, 0x6d, 0x65, 0x12, 0x39, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x02, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c,\n\t0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x12,\n\t0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x6b, 0x65,\n\t0x79, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20,\n\t0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x64, 0x1a, 0x10, 0x0a, 0x0e, 0x52,\n\t0x65, 0x73, 0x75, 0x6c, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x1a, 0xcd, 0x02,\n\t0x0a, 0x11, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72,\n\t0x65, 0x73, 0x73, 0x12, 0x56, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x18, 0x01, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,\n\t0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x67, 0x65, 0x49,\n\t0x6e, 0x66, 0x6f, 0x52, 0x06, 0x73, 0x74, 0x61, 0x67, 0x65, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x6e,\n\t0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x66, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x74, 0x61, 0x73, 0x6b,\n\t0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x49, 0x6e, 0x66, 0x6c,\n\t0x69, 0x67, 0x68, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x1a, 0xb1, 0x01, 0x0a, 0x09, 0x53, 0x74,\n\t0x61, 0x67, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x67, 0x65,\n\t0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x73, 0x74, 0x61, 0x67, 0x65,\n\t0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x75, 0x6d, 0x5f, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x6e, 0x75, 0x6d, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12,\n\t0x2e, 0x0a, 0x13, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64,\n\t0x5f, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x6e, 0x75,\n\t0x6d, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x54, 0x61, 0x73, 0x6b, 0x73, 0x12,\n\t0x28, 0x0a, 0x10, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x72,\n\t0x65, 0x61, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x42, 0x79, 0x74, 0x65, 0x73, 0x52, 0x65, 0x61, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x6f, 0x6e,\n\t0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x64, 0x6f, 0x6e, 0x65, 0x42, 0x0f, 0x0a,\n\t0x0d, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x41,\n\t0x0a, 0x08, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,\n\t0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x19, 0x0a, 0x05,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x76, 0x61, 0x6c, 0x75,\n\t0x65, 0x22, 0xaf, 0x09, 0x0a, 0x0d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75,\n\t0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69,\n\t0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73,\n\t0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64,\n\t0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x08, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x00, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65,\n\t0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0c, 0x75, 0x73,\n\t0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x75, 0x73,\n\t0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x44, 0x0a, 0x09, 0x6f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e,\n\t0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12,\n\t0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04,\n\t0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79,\n\t0x70, 0x65, 0x88, 0x01, 0x01, 0x1a, 0xf2, 0x03, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, 0x03, 0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53,\n\t0x65, 0x74, 0x48, 0x00, 0x52, 0x03, 0x73, 0x65, 0x74, 0x12, 0x34, 0x0a, 0x03, 0x67, 0x65, 0x74,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71,\n\t0x75, 0x65, 0x73, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x48, 0x00, 0x52, 0x03, 0x67, 0x65, 0x74, 0x12,\n\t0x57, 0x0a, 0x10, 0x67, 0x65, 0x74, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x64, 0x65, 0x66, 0x61,\n\t0x75, 0x6c, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67,\n\t0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x57, 0x69, 0x74, 0x68, 0x44,\n\t0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x67, 0x65, 0x74, 0x57, 0x69, 0x74,\n\t0x68, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x47, 0x0a, 0x0a, 0x67, 0x65, 0x74, 0x5f,\n\t0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e,\n\t0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x09, 0x67, 0x65, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x12, 0x3e, 0x0a, 0x07, 0x67, 0x65, 0x74, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,\n\t0x2e, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x06, 0x67, 0x65, 0x74, 0x41, 0x6c,\n\t0x6c, 0x12, 0x3a, 0x0a, 0x05, 0x75, 0x6e, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x55,\n\t0x6e, 0x73, 0x65, 0x74, 0x48, 0x00, 0x52, 0x05, 0x75, 0x6e, 0x73, 0x65, 0x74, 0x12, 0x50, 0x0a,\n\t0x0d, 0x69, 0x73, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x07,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65,\n\t0x73, 0x74, 0x2e, 0x49, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x48,\n\t0x00, 0x52, 0x0c, 0x69, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x42,\n\t0x09, 0x0a, 0x07, 0x6f, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x1a, 0x5c, 0x0a, 0x03, 0x53, 0x65,\n\t0x74, 0x12, 0x2d, 0x0a, 0x05, 0x70, 0x61, 0x69, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x70, 0x61, 0x69, 0x72, 0x73,\n\t0x12, 0x1b, 0x0a, 0x06, 0x73, 0x69, 0x6c, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08,\n\t0x48, 0x00, 0x52, 0x06, 0x73, 0x69, 0x6c, 0x65, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a,\n\t0x07, 0x5f, 0x73, 0x69, 0x6c, 0x65, 0x6e, 0x74, 0x1a, 0x19, 0x0a, 0x03, 0x47, 0x65, 0x74, 0x12,\n\t0x12, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x6b,\n\t0x65, 0x79, 0x73, 0x1a, 0x3f, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x57, 0x69, 0x74, 0x68, 0x44, 0x65,\n\t0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x70, 0x61, 0x69, 0x72, 0x73, 0x18, 0x01,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x70,\n\t0x61, 0x69, 0x72, 0x73, 0x1a, 0x1f, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52,\n\t0x04, 0x6b, 0x65, 0x79, 0x73, 0x1a, 0x30, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x41, 0x6c, 0x6c, 0x12,\n\t0x1b, 0x0a, 0x06, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x06, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a, 0x07,\n\t0x5f, 0x70, 0x72, 0x65, 0x66, 0x69, 0x78, 0x1a, 0x1b, 0x0a, 0x05, 0x55, 0x6e, 0x73, 0x65, 0x74,\n\t0x12, 0x12, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04,\n\t0x6b, 0x65, 0x79, 0x73, 0x1a, 0x22, 0x0a, 0x0c, 0x49, 0x73, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69,\n\t0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x01, 0x20, 0x03,\n\t0x28, 0x09, 0x52, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69,\n\t0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72,\n\t0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x22, 0xaf, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65,\n\t0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,\n\t0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,\n\t0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64,\n\t0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x2d, 0x0a, 0x05, 0x70, 0x61,\n\t0x69, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c,\n\t0x75, 0x65, 0x52, 0x05, 0x70, 0x61, 0x69, 0x72, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x61, 0x72,\n\t0x6e, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x77, 0x61, 0x72,\n\t0x6e, 0x69, 0x6e, 0x67, 0x73, 0x22, 0xea, 0x07, 0x0a, 0x13, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74,\n\t0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a,\n\t0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x3d, 0x0a, 0x0c,\n\t0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b,\n\t0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x56, 0x0a, 0x26, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73,\n\t0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x21, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72,\n\t0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64,\n\t0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79,\n\t0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65,\n\t0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x40, 0x0a, 0x05, 0x62, 0x61, 0x74,\n\t0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69,\n\t0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x61, 0x74,\n\t0x63, 0x68, 0x48, 0x00, 0x52, 0x05, 0x62, 0x61, 0x74, 0x63, 0x68, 0x12, 0x5a, 0x0a, 0x0b, 0x62,\n\t0x65, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71,\n\t0x75, 0x65, 0x73, 0x74, 0x2e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65,\n\t0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x62, 0x65, 0x67,\n\t0x69, 0x6e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x48, 0x0a, 0x05, 0x63, 0x68, 0x75, 0x6e, 0x6b,\n\t0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61,\n\t0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66,\n\t0x61, 0x63, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x63, 0x68, 0x75, 0x6e,\n\t0x6b, 0x1a, 0x35, 0x0a, 0x0d, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68, 0x75,\n\t0x6e, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c,\n\t0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x72, 0x63, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x03, 0x52, 0x03, 0x63, 0x72, 0x63, 0x1a, 0x6f, 0x0a, 0x13, 0x53, 0x69, 0x6e, 0x67,\n\t0x6c, 0x65, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12,\n\t0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,\n\t0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65,\n\t0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68,\n\t0x75, 0x6e, 0x6b, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5d, 0x0a, 0x05, 0x42, 0x61, 0x74,\n\t0x63, 0x68, 0x12, 0x54, 0x0a, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18,\n\t0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,\n\t0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65,\n\t0x43, 0x68, 0x75, 0x6e, 0x6b, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x09, 0x61,\n\t0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x1a, 0xc1, 0x01, 0x0a, 0x14, 0x42, 0x65, 0x67,\n\t0x69, 0x6e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x65, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,\n\t0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62,\n\t0x79, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x74, 0x6f, 0x74, 0x61,\n\t0x6c, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x75, 0x6d, 0x5f, 0x63, 0x68,\n\t0x75, 0x6e, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x43,\n\t0x68, 0x75, 0x6e, 0x6b, 0x73, 0x12, 0x55, 0x0a, 0x0d, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c,\n\t0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x64, 0x64,\n\t0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,\n\t0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x0c,\n\t0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x42, 0x09, 0x0a, 0x07,\n\t0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65,\n\t0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79,\n\t0x70, 0x65, 0x22, 0x90, 0x02, 0x0a, 0x14, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61,\n\t0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65,\n\t0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,\n\t0x51, 0x0a, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03,\n\t0x28, 0x0b, 0x32, 0x33, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52,\n\t0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74,\n\t0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,\n\t0x74, 0x73, 0x1a, 0x51, 0x0a, 0x0f, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x75,\n\t0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x73, 0x5f,\n\t0x63, 0x72, 0x63, 0x5f, 0x73, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x66, 0x75, 0x6c, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x43, 0x72, 0x63, 0x53, 0x75, 0x63, 0x63, 0x65,\n\t0x73, 0x73, 0x66, 0x75, 0x6c, 0x22, 0xc6, 0x02, 0x0a, 0x17, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61,\n\t0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,\n\t0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64,\n\t0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72,\n\t0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f,\n\t0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72,\n\t0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55,\n\t0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72,\n\t0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e,\n\t0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a,\n\t0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x14, 0x0a,\n\t0x05, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x61,\n\t0x6d, 0x65, 0x73, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f,\n\t0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73,\n\t0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e,\n\t0x0a, 0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xe0,\n\t0x02, 0x0a, 0x18, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75,\n\t0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65,\n\t0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,\n\t0x51, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x35, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,\n\t0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75,\n\t0x73, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73,\n\t0x65, 0x73, 0x1a, 0x73, 0x0a, 0x0d, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x45, 0x6e,\n\t0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61,\n\t0x74, 0x75, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x72,\n\t0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x28, 0x0a, 0x0e, 0x41, 0x72, 0x74, 0x69, 0x66,\n\t0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x65, 0x78, 0x69,\n\t0x73, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x65, 0x78, 0x69, 0x73, 0x74,\n\t0x73, 0x22, 0xdb, 0x04, 0x0a, 0x10, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,\n\t0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,\n\t0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,\n\t0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f,\n\t0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64,\n\t0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a,\n\t0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52,\n\t0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x24, 0x0a, 0x0b,\n\t0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x02, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88,\n\t0x01, 0x01, 0x12, 0x54, 0x0a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x5f,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72,\n\t0x72, 0x75, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x74, 0x65,\n\t0x72, 0x72, 0x75, 0x70, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x74, 0x65, 0x72,\n\t0x72, 0x75, 0x70, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x6f, 0x70, 0x65, 0x72,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x67, 0x12,\n\t0x23, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18,\n\t0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x49, 0x64, 0x22, 0x80, 0x01, 0x0a, 0x0d, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75,\n\t0x70, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x1a, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x52,\n\t0x55, 0x50, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49,\n\t0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x52,\n\t0x55, 0x50, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, 0x4c, 0x4c, 0x10, 0x01, 0x12, 0x16,\n\t0x0a, 0x12, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x52, 0x55, 0x50, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,\n\t0x5f, 0x54, 0x41, 0x47, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x49, 0x4e, 0x54, 0x45, 0x52, 0x52,\n\t0x55, 0x50, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49,\n\t0x4f, 0x4e, 0x5f, 0x49, 0x44, 0x10, 0x03, 0x42, 0x0b, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x65, 0x72,\n\t0x72, 0x75, 0x70, 0x74, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,\n\t0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f,\n\t0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42,\n\t0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22,\n\t0x90, 0x01, 0x0a, 0x11, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73,\n\t0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65,\n\t0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x69, 0x6e, 0x74,\n\t0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03,\n\t0x28, 0x09, 0x52, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x65, 0x64, 0x49,\n\t0x64, 0x73, 0x22, 0x35, 0x0a, 0x0f, 0x52, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63,\n\t0x68, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x72, 0x65, 0x61,\n\t0x74, 0x74, 0x61, 0x63, 0x68, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x96, 0x03, 0x0a, 0x16, 0x52, 0x65,\n\t0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71,\n\t0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62,\n\t0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69,\n\t0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73,\n\t0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0c, 0x75,\n\t0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x75,\n\t0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x70,\n\t0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x24, 0x0a,\n\t0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65,\n\t0x88, 0x01, 0x01, 0x12, 0x2d, 0x0a, 0x10, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52,\n\t0x0e, 0x6c, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x49, 0x64, 0x88,\n\t0x01, 0x01, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62,\n\t0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69,\n\t0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a,\n\t0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42, 0x13, 0x0a,\n\t0x11, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x5f,\n\t0x69, 0x64, 0x22, 0xc9, 0x04, 0x0a, 0x15, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x45, 0x78,\n\t0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a,\n\t0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73,\n\t0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x21, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72,\n\t0x76, 0x65, 0x72, 0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64,\n\t0x88, 0x01, 0x01, 0x12, 0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74,\n\t0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f,\n\t0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65,\n\t0x78, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0a, 0x63, 0x6c,\n\t0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x52, 0x0a, 0x0b, 0x72,\n\t0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x2f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x6c,\n\t0x6c, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x41, 0x6c, 0x6c, 0x12,\n\t0x58, 0x0a, 0x0d, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x5f, 0x75, 0x6e, 0x74, 0x69, 0x6c,\n\t0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x45, 0x78,\n\t0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x65, 0x61, 0x73, 0x65, 0x55, 0x6e, 0x74, 0x69, 0x6c, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x65, 0x6c,\n\t0x65, 0x61, 0x73, 0x65, 0x55, 0x6e, 0x74, 0x69, 0x6c, 0x1a, 0x0c, 0x0a, 0x0a, 0x52, 0x65, 0x6c,\n\t0x65, 0x61, 0x73, 0x65, 0x41, 0x6c, 0x6c, 0x1a, 0x2f, 0x0a, 0x0c, 0x52, 0x65, 0x6c, 0x65, 0x61,\n\t0x73, 0x65, 0x55, 0x6e, 0x74, 0x69, 0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x73, 0x70, 0x6f,\n\t0x6e, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65,\n\t0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x49, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x6c, 0x65,\n\t0x61, 0x73, 0x65, 0x42, 0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f,\n\t0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73,\n\t0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e,\n\t0x0a, 0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xa5,\n\t0x01, 0x0a, 0x16, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74,\n\t0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,\n\t0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x26, 0x0a,\n\t0x0c, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x49, 0x64, 0x88, 0x01, 0x01, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x22, 0xd4, 0x01, 0x0a, 0x15, 0x52, 0x65, 0x6c, 0x65, 0x61,\n\t0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,\n\t0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,\n\t0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78,\n\t0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x24,\n\t0x0a, 0x0b, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,\n\t0x65, 0x88, 0x01, 0x01, 0x12, 0x27, 0x0a, 0x0f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x72, 0x65,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x61,\n\t0x6c, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x42, 0x0e, 0x0a,\n\t0x0c, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x6c, 0x0a,\n\t0x16, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52,\n\t0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,\n\t0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x69,\n\t0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0xcc, 0x02, 0x0a, 0x18,\n\t0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c,\n\t0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x56, 0x0a, 0x26, 0x63, 0x6c, 0x69, 0x65, 0x6e,\n\t0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65,\n\t0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69,\n\t0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x21, 0x63, 0x6c, 0x69, 0x65, 0x6e,\n\t0x74, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53,\n\t0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12,\n\t0x3d, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78,\n\t0x74, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x19,\n\t0x0a, 0x08, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x07, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0b, 0x63, 0x6c, 0x69,\n\t0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01,\n\t0x52, 0x0a, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x42,\n\t0x29, 0x0a, 0x27, 0x5f, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72,\n\t0x76, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f,\n\t0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63,\n\t0x6c, 0x69, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x93, 0x0c, 0x0a, 0x19, 0x46,\n\t0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73,\n\t0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x16, 0x73, 0x65, 0x72, 0x76,\n\t0x65, 0x72, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72,\n\t0x53, 0x69, 0x64, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a,\n\t0x0a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x0e,\n\t0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x78, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x6f, 0x6f, 0x74, 0x45, 0x72, 0x72, 0x6f,\n\t0x72, 0x49, 0x64, 0x78, 0x88, 0x01, 0x01, 0x12, 0x46, 0x0a, 0x06, 0x65, 0x72, 0x72, 0x6f, 0x72,\n\t0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72,\n\t0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x2e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x52, 0x06, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x73, 0x1a,\n\t0xae, 0x01, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x6c,\n\t0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x63, 0x6c, 0x61, 0x72, 0x69,\n\t0x6e, 0x67, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,\n\t0x64, 0x65, 0x63, 0x6c, 0x61, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x12, 0x1f,\n\t0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12,\n\t0x20, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01,\n\t0x01, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72,\n\t0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x6c, 0x69, 0x6e, 0x65, 0x4e, 0x75, 0x6d, 0x62,\n\t0x65, 0x72, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x1a, 0xf0, 0x02, 0x0a, 0x0c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78,\n\t0x74, 0x12, 0x64, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x74, 0x79, 0x70,\n\t0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x41, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72,\n\t0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x2e, 0x43,\n\t0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x74,\n\t0x65, 0x78, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6f, 0x62, 0x6a, 0x65, 0x63,\n\t0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f, 0x62,\n\t0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6f, 0x62, 0x6a, 0x65,\n\t0x63, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f,\n\t0x62, 0x6a, 0x65, 0x63, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61,\n\t0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a,\n\t0x73, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74,\n\t0x6f, 0x70, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09,\n\t0x73, 0x74, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x1a, 0x0a, 0x08, 0x66, 0x72, 0x61,\n\t0x67, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x72, 0x61,\n\t0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x69,\n\t0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x61, 0x6c, 0x6c, 0x53, 0x69,\n\t0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x07, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x22, 0x25, 0x0a, 0x0b,\n\t0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x53,\n\t0x51, 0x4c, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x41, 0x54, 0x41, 0x46, 0x52, 0x41, 0x4d,\n\t0x45, 0x10, 0x01, 0x1a, 0x99, 0x03, 0x0a, 0x0e, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x54, 0x68, 0x72,\n\t0x6f, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x24, 0x0a, 0x0b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f,\n\t0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0a, 0x65,\n\t0x72, 0x72, 0x6f, 0x72, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x88, 0x01, 0x01, 0x12, 0x7d, 0x0a, 0x12,\n\t0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65,\n\t0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72,\n\t0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,\n\t0x73, 0x65, 0x2e, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x77, 0x61, 0x62, 0x6c,\n\t0x65, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74,\n\t0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,\n\t0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, 0x5c, 0x0a, 0x0e, 0x71,\n\t0x75, 0x65, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x73, 0x18, 0x03, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65,\n\t0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0d, 0x71, 0x75, 0x65, 0x72,\n\t0x79, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x73, 0x12, 0x20, 0x0a, 0x09, 0x73, 0x71, 0x6c,\n\t0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x08,\n\t0x73, 0x71, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x88, 0x01, 0x01, 0x1a, 0x44, 0x0a, 0x16, 0x4d,\n\t0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73,\n\t0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,\n\t0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6c, 0x61, 0x73,\n\t0x73, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x73, 0x71, 0x6c, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x1a,\n\t0xdb, 0x02, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x30, 0x0a, 0x14, 0x65, 0x72, 0x72,\n\t0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x68, 0x69, 0x65, 0x72, 0x61, 0x72, 0x63, 0x68,\n\t0x79, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x54, 0x79,\n\t0x70, 0x65, 0x48, 0x69, 0x65, 0x72, 0x61, 0x72, 0x63, 0x68, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x6d,\n\t0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65,\n\t0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x5b, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x5f, 0x74,\n\t0x72, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68,\n\t0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45,\n\t0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61,\n\t0x63, 0x65, 0x12, 0x20, 0x0a, 0x09, 0x63, 0x61, 0x75, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x78, 0x18,\n\t0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x08, 0x63, 0x61, 0x75, 0x73, 0x65, 0x49, 0x64,\n\t0x78, 0x88, 0x01, 0x01, 0x12, 0x65, 0x0a, 0x0f, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x5f, 0x74, 0x68,\n\t0x72, 0x6f, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65,\n\t0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52,\n\t0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x54, 0x68, 0x72,\n\t0x6f, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x01, 0x52, 0x0e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x54,\n\t0x68, 0x72, 0x6f, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0c, 0x0a, 0x0a, 0x5f,\n\t0x63, 0x61, 0x75, 0x73, 0x65, 0x5f, 0x69, 0x64, 0x78, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x11, 0x0a,\n\t0x0f, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x78,\n\t0x22, 0x5a, 0x0a, 0x17, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x3f, 0x0a, 0x08, 0x72,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x61,\n\t0x63, 0x68, 0x65, 0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x32, 0xb2, 0x07, 0x0a,\n\t0x13, 0x53, 0x70, 0x61, 0x72, 0x6b, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x53, 0x65, 0x72,\n\t0x76, 0x69, 0x63, 0x65, 0x12, 0x58, 0x0a, 0x0b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50,\n\t0x6c, 0x61, 0x6e, 0x12, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c,\n\t0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x56,\n\t0x0a, 0x0b, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x21, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x6e,\n\t0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,\n\t0x1a, 0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a, 0x06, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67,\n\t0x12, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43,\n\t0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,\n\t0x5b, 0x0a, 0x0c, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x12,\n\t0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75,\n\t0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x41, 0x64, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73,\n\t0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x12, 0x63, 0x0a, 0x0e,\n\t0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x26,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41,\n\t0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x52,\n\t0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53,\n\t0x74, 0x61, 0x74, 0x75, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,\n\t0x00, 0x12, 0x50, 0x0a, 0x09, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x12, 0x1f,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x49,\n\t0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,\n\t0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x49, 0x6e, 0x74, 0x65, 0x72, 0x72, 0x75, 0x70, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0f, 0x52, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x45,\n\t0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x12, 0x25, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x74, 0x74, 0x61, 0x63, 0x68, 0x45,\n\t0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x65, 0x63, 0x75, 0x74, 0x65, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,\n\t0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x5f, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65,\n\t0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x12, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x45,\n\t0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x6c, 0x65, 0x61, 0x73, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70,\n\t0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73,\n\t0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65,\n\t0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x11, 0x46, 0x65, 0x74, 0x63, 0x68,\n\t0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74,\n\t0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65,\n\t0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x45, 0x72, 0x72, 0x6f, 0x72,\n\t0x44, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,\n\t0x00, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x72,\n\t0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f,\n\t0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,\n\t0x33,\n}\n\nvar (\n\tfile_spark_connect_base_proto_rawDescOnce sync.Once\n\tfile_spark_connect_base_proto_rawDescData = file_spark_connect_base_proto_rawDesc\n)\n\nfunc file_spark_connect_base_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_base_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_base_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_base_proto_rawDescData)\n\t})\n\treturn file_spark_connect_base_proto_rawDescData\n}\n\nvar file_spark_connect_base_proto_enumTypes = make([]protoimpl.EnumInfo, 3)\nvar file_spark_connect_base_proto_msgTypes = make([]protoimpl.MessageInfo, 85)\nvar file_spark_connect_base_proto_goTypes = []interface{}{\n\t(AnalyzePlanRequest_Explain_ExplainMode)(0),             // 0: spark.connect.AnalyzePlanRequest.Explain.ExplainMode\n\t(InterruptRequest_InterruptType)(0),                     // 1: spark.connect.InterruptRequest.InterruptType\n\t(FetchErrorDetailsResponse_QueryContext_ContextType)(0), // 2: spark.connect.FetchErrorDetailsResponse.QueryContext.ContextType\n\t(*Plan)(nil),                                     // 3: spark.connect.Plan\n\t(*UserContext)(nil),                              // 4: spark.connect.UserContext\n\t(*AnalyzePlanRequest)(nil),                       // 5: spark.connect.AnalyzePlanRequest\n\t(*AnalyzePlanResponse)(nil),                      // 6: spark.connect.AnalyzePlanResponse\n\t(*ExecutePlanRequest)(nil),                       // 7: spark.connect.ExecutePlanRequest\n\t(*ExecutePlanResponse)(nil),                      // 8: spark.connect.ExecutePlanResponse\n\t(*KeyValue)(nil),                                 // 9: spark.connect.KeyValue\n\t(*ConfigRequest)(nil),                            // 10: spark.connect.ConfigRequest\n\t(*ConfigResponse)(nil),                           // 11: spark.connect.ConfigResponse\n\t(*AddArtifactsRequest)(nil),                      // 12: spark.connect.AddArtifactsRequest\n\t(*AddArtifactsResponse)(nil),                     // 13: spark.connect.AddArtifactsResponse\n\t(*ArtifactStatusesRequest)(nil),                  // 14: spark.connect.ArtifactStatusesRequest\n\t(*ArtifactStatusesResponse)(nil),                 // 15: spark.connect.ArtifactStatusesResponse\n\t(*InterruptRequest)(nil),                         // 16: spark.connect.InterruptRequest\n\t(*InterruptResponse)(nil),                        // 17: spark.connect.InterruptResponse\n\t(*ReattachOptions)(nil),                          // 18: spark.connect.ReattachOptions\n\t(*ReattachExecuteRequest)(nil),                   // 19: spark.connect.ReattachExecuteRequest\n\t(*ReleaseExecuteRequest)(nil),                    // 20: spark.connect.ReleaseExecuteRequest\n\t(*ReleaseExecuteResponse)(nil),                   // 21: spark.connect.ReleaseExecuteResponse\n\t(*ReleaseSessionRequest)(nil),                    // 22: spark.connect.ReleaseSessionRequest\n\t(*ReleaseSessionResponse)(nil),                   // 23: spark.connect.ReleaseSessionResponse\n\t(*FetchErrorDetailsRequest)(nil),                 // 24: spark.connect.FetchErrorDetailsRequest\n\t(*FetchErrorDetailsResponse)(nil),                // 25: spark.connect.FetchErrorDetailsResponse\n\t(*CheckpointCommandResult)(nil),                  // 26: spark.connect.CheckpointCommandResult\n\t(*AnalyzePlanRequest_Schema)(nil),                // 27: spark.connect.AnalyzePlanRequest.Schema\n\t(*AnalyzePlanRequest_Explain)(nil),               // 28: spark.connect.AnalyzePlanRequest.Explain\n\t(*AnalyzePlanRequest_TreeString)(nil),            // 29: spark.connect.AnalyzePlanRequest.TreeString\n\t(*AnalyzePlanRequest_IsLocal)(nil),               // 30: spark.connect.AnalyzePlanRequest.IsLocal\n\t(*AnalyzePlanRequest_IsStreaming)(nil),           // 31: spark.connect.AnalyzePlanRequest.IsStreaming\n\t(*AnalyzePlanRequest_InputFiles)(nil),            // 32: spark.connect.AnalyzePlanRequest.InputFiles\n\t(*AnalyzePlanRequest_SparkVersion)(nil),          // 33: spark.connect.AnalyzePlanRequest.SparkVersion\n\t(*AnalyzePlanRequest_DDLParse)(nil),              // 34: spark.connect.AnalyzePlanRequest.DDLParse\n\t(*AnalyzePlanRequest_SameSemantics)(nil),         // 35: spark.connect.AnalyzePlanRequest.SameSemantics\n\t(*AnalyzePlanRequest_SemanticHash)(nil),          // 36: spark.connect.AnalyzePlanRequest.SemanticHash\n\t(*AnalyzePlanRequest_Persist)(nil),               // 37: spark.connect.AnalyzePlanRequest.Persist\n\t(*AnalyzePlanRequest_Unpersist)(nil),             // 38: spark.connect.AnalyzePlanRequest.Unpersist\n\t(*AnalyzePlanRequest_GetStorageLevel)(nil),       // 39: spark.connect.AnalyzePlanRequest.GetStorageLevel\n\t(*AnalyzePlanRequest_JsonToDDL)(nil),             // 40: spark.connect.AnalyzePlanRequest.JsonToDDL\n\t(*AnalyzePlanResponse_Schema)(nil),               // 41: spark.connect.AnalyzePlanResponse.Schema\n\t(*AnalyzePlanResponse_Explain)(nil),              // 42: spark.connect.AnalyzePlanResponse.Explain\n\t(*AnalyzePlanResponse_TreeString)(nil),           // 43: spark.connect.AnalyzePlanResponse.TreeString\n\t(*AnalyzePlanResponse_IsLocal)(nil),              // 44: spark.connect.AnalyzePlanResponse.IsLocal\n\t(*AnalyzePlanResponse_IsStreaming)(nil),          // 45: spark.connect.AnalyzePlanResponse.IsStreaming\n\t(*AnalyzePlanResponse_InputFiles)(nil),           // 46: spark.connect.AnalyzePlanResponse.InputFiles\n\t(*AnalyzePlanResponse_SparkVersion)(nil),         // 47: spark.connect.AnalyzePlanResponse.SparkVersion\n\t(*AnalyzePlanResponse_DDLParse)(nil),             // 48: spark.connect.AnalyzePlanResponse.DDLParse\n\t(*AnalyzePlanResponse_SameSemantics)(nil),        // 49: spark.connect.AnalyzePlanResponse.SameSemantics\n\t(*AnalyzePlanResponse_SemanticHash)(nil),         // 50: spark.connect.AnalyzePlanResponse.SemanticHash\n\t(*AnalyzePlanResponse_Persist)(nil),              // 51: spark.connect.AnalyzePlanResponse.Persist\n\t(*AnalyzePlanResponse_Unpersist)(nil),            // 52: spark.connect.AnalyzePlanResponse.Unpersist\n\t(*AnalyzePlanResponse_GetStorageLevel)(nil),      // 53: spark.connect.AnalyzePlanResponse.GetStorageLevel\n\t(*AnalyzePlanResponse_JsonToDDL)(nil),            // 54: spark.connect.AnalyzePlanResponse.JsonToDDL\n\t(*ExecutePlanRequest_RequestOption)(nil),         // 55: spark.connect.ExecutePlanRequest.RequestOption\n\t(*ExecutePlanResponse_SqlCommandResult)(nil),     // 56: spark.connect.ExecutePlanResponse.SqlCommandResult\n\t(*ExecutePlanResponse_ArrowBatch)(nil),           // 57: spark.connect.ExecutePlanResponse.ArrowBatch\n\t(*ExecutePlanResponse_Metrics)(nil),              // 58: spark.connect.ExecutePlanResponse.Metrics\n\t(*ExecutePlanResponse_ObservedMetrics)(nil),      // 59: spark.connect.ExecutePlanResponse.ObservedMetrics\n\t(*ExecutePlanResponse_ResultComplete)(nil),       // 60: spark.connect.ExecutePlanResponse.ResultComplete\n\t(*ExecutePlanResponse_ExecutionProgress)(nil),    // 61: spark.connect.ExecutePlanResponse.ExecutionProgress\n\t(*ExecutePlanResponse_Metrics_MetricObject)(nil), // 62: spark.connect.ExecutePlanResponse.Metrics.MetricObject\n\t(*ExecutePlanResponse_Metrics_MetricValue)(nil),  // 63: spark.connect.ExecutePlanResponse.Metrics.MetricValue\n\tnil, // 64: spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry\n\t(*ExecutePlanResponse_ExecutionProgress_StageInfo)(nil), // 65: spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfo\n\t(*ConfigRequest_Operation)(nil),                         // 66: spark.connect.ConfigRequest.Operation\n\t(*ConfigRequest_Set)(nil),                               // 67: spark.connect.ConfigRequest.Set\n\t(*ConfigRequest_Get)(nil),                               // 68: spark.connect.ConfigRequest.Get\n\t(*ConfigRequest_GetWithDefault)(nil),                    // 69: spark.connect.ConfigRequest.GetWithDefault\n\t(*ConfigRequest_GetOption)(nil),                         // 70: spark.connect.ConfigRequest.GetOption\n\t(*ConfigRequest_GetAll)(nil),                            // 71: spark.connect.ConfigRequest.GetAll\n\t(*ConfigRequest_Unset)(nil),                             // 72: spark.connect.ConfigRequest.Unset\n\t(*ConfigRequest_IsModifiable)(nil),                      // 73: spark.connect.ConfigRequest.IsModifiable\n\t(*AddArtifactsRequest_ArtifactChunk)(nil),               // 74: spark.connect.AddArtifactsRequest.ArtifactChunk\n\t(*AddArtifactsRequest_SingleChunkArtifact)(nil),         // 75: spark.connect.AddArtifactsRequest.SingleChunkArtifact\n\t(*AddArtifactsRequest_Batch)(nil),                       // 76: spark.connect.AddArtifactsRequest.Batch\n\t(*AddArtifactsRequest_BeginChunkedArtifact)(nil),        // 77: spark.connect.AddArtifactsRequest.BeginChunkedArtifact\n\t(*AddArtifactsResponse_ArtifactSummary)(nil),            // 78: spark.connect.AddArtifactsResponse.ArtifactSummary\n\tnil, // 79: spark.connect.ArtifactStatusesResponse.StatusesEntry\n\t(*ArtifactStatusesResponse_ArtifactStatus)(nil),     // 80: spark.connect.ArtifactStatusesResponse.ArtifactStatus\n\t(*ReleaseExecuteRequest_ReleaseAll)(nil),            // 81: spark.connect.ReleaseExecuteRequest.ReleaseAll\n\t(*ReleaseExecuteRequest_ReleaseUntil)(nil),          // 82: spark.connect.ReleaseExecuteRequest.ReleaseUntil\n\t(*FetchErrorDetailsResponse_StackTraceElement)(nil), // 83: spark.connect.FetchErrorDetailsResponse.StackTraceElement\n\t(*FetchErrorDetailsResponse_QueryContext)(nil),      // 84: spark.connect.FetchErrorDetailsResponse.QueryContext\n\t(*FetchErrorDetailsResponse_SparkThrowable)(nil),    // 85: spark.connect.FetchErrorDetailsResponse.SparkThrowable\n\t(*FetchErrorDetailsResponse_Error)(nil),             // 86: spark.connect.FetchErrorDetailsResponse.Error\n\tnil,                                                 // 87: spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntry\n\t(*Relation)(nil),                                    // 88: spark.connect.Relation\n\t(*Command)(nil),                                     // 89: spark.connect.Command\n\t(*anypb.Any)(nil),                                   // 90: google.protobuf.Any\n\t(*WriteStreamOperationStartResult)(nil),             // 91: spark.connect.WriteStreamOperationStartResult\n\t(*StreamingQueryCommandResult)(nil),                 // 92: spark.connect.StreamingQueryCommandResult\n\t(*GetResourcesCommandResult)(nil),                   // 93: spark.connect.GetResourcesCommandResult\n\t(*StreamingQueryManagerCommandResult)(nil),          // 94: spark.connect.StreamingQueryManagerCommandResult\n\t(*StreamingQueryListenerEventsResult)(nil),          // 95: spark.connect.StreamingQueryListenerEventsResult\n\t(*CreateResourceProfileCommandResult)(nil),          // 96: spark.connect.CreateResourceProfileCommandResult\n\t(*MlCommandResult)(nil),                             // 97: spark.connect.MlCommandResult\n\t(*PipelineEventResult)(nil),                         // 98: spark.connect.PipelineEventResult\n\t(*PipelineCommandResult)(nil),                       // 99: spark.connect.PipelineCommandResult\n\t(*DataType)(nil),                                    // 100: spark.connect.DataType\n\t(*CachedRemoteRelation)(nil),                        // 101: spark.connect.CachedRemoteRelation\n\t(*StorageLevel)(nil),                                // 102: spark.connect.StorageLevel\n\t(*Expression_Literal)(nil),                          // 103: spark.connect.Expression.Literal\n}\nvar file_spark_connect_base_proto_depIdxs = []int32{\n\t88,  // 0: spark.connect.Plan.root:type_name -> spark.connect.Relation\n\t89,  // 1: spark.connect.Plan.command:type_name -> spark.connect.Command\n\t90,  // 2: spark.connect.UserContext.extensions:type_name -> google.protobuf.Any\n\t4,   // 3: spark.connect.AnalyzePlanRequest.user_context:type_name -> spark.connect.UserContext\n\t27,  // 4: spark.connect.AnalyzePlanRequest.schema:type_name -> spark.connect.AnalyzePlanRequest.Schema\n\t28,  // 5: spark.connect.AnalyzePlanRequest.explain:type_name -> spark.connect.AnalyzePlanRequest.Explain\n\t29,  // 6: spark.connect.AnalyzePlanRequest.tree_string:type_name -> spark.connect.AnalyzePlanRequest.TreeString\n\t30,  // 7: spark.connect.AnalyzePlanRequest.is_local:type_name -> spark.connect.AnalyzePlanRequest.IsLocal\n\t31,  // 8: spark.connect.AnalyzePlanRequest.is_streaming:type_name -> spark.connect.AnalyzePlanRequest.IsStreaming\n\t32,  // 9: spark.connect.AnalyzePlanRequest.input_files:type_name -> spark.connect.AnalyzePlanRequest.InputFiles\n\t33,  // 10: spark.connect.AnalyzePlanRequest.spark_version:type_name -> spark.connect.AnalyzePlanRequest.SparkVersion\n\t34,  // 11: spark.connect.AnalyzePlanRequest.ddl_parse:type_name -> spark.connect.AnalyzePlanRequest.DDLParse\n\t35,  // 12: spark.connect.AnalyzePlanRequest.same_semantics:type_name -> spark.connect.AnalyzePlanRequest.SameSemantics\n\t36,  // 13: spark.connect.AnalyzePlanRequest.semantic_hash:type_name -> spark.connect.AnalyzePlanRequest.SemanticHash\n\t37,  // 14: spark.connect.AnalyzePlanRequest.persist:type_name -> spark.connect.AnalyzePlanRequest.Persist\n\t38,  // 15: spark.connect.AnalyzePlanRequest.unpersist:type_name -> spark.connect.AnalyzePlanRequest.Unpersist\n\t39,  // 16: spark.connect.AnalyzePlanRequest.get_storage_level:type_name -> spark.connect.AnalyzePlanRequest.GetStorageLevel\n\t40,  // 17: spark.connect.AnalyzePlanRequest.json_to_ddl:type_name -> spark.connect.AnalyzePlanRequest.JsonToDDL\n\t41,  // 18: spark.connect.AnalyzePlanResponse.schema:type_name -> spark.connect.AnalyzePlanResponse.Schema\n\t42,  // 19: spark.connect.AnalyzePlanResponse.explain:type_name -> spark.connect.AnalyzePlanResponse.Explain\n\t43,  // 20: spark.connect.AnalyzePlanResponse.tree_string:type_name -> spark.connect.AnalyzePlanResponse.TreeString\n\t44,  // 21: spark.connect.AnalyzePlanResponse.is_local:type_name -> spark.connect.AnalyzePlanResponse.IsLocal\n\t45,  // 22: spark.connect.AnalyzePlanResponse.is_streaming:type_name -> spark.connect.AnalyzePlanResponse.IsStreaming\n\t46,  // 23: spark.connect.AnalyzePlanResponse.input_files:type_name -> spark.connect.AnalyzePlanResponse.InputFiles\n\t47,  // 24: spark.connect.AnalyzePlanResponse.spark_version:type_name -> spark.connect.AnalyzePlanResponse.SparkVersion\n\t48,  // 25: spark.connect.AnalyzePlanResponse.ddl_parse:type_name -> spark.connect.AnalyzePlanResponse.DDLParse\n\t49,  // 26: spark.connect.AnalyzePlanResponse.same_semantics:type_name -> spark.connect.AnalyzePlanResponse.SameSemantics\n\t50,  // 27: spark.connect.AnalyzePlanResponse.semantic_hash:type_name -> spark.connect.AnalyzePlanResponse.SemanticHash\n\t51,  // 28: spark.connect.AnalyzePlanResponse.persist:type_name -> spark.connect.AnalyzePlanResponse.Persist\n\t52,  // 29: spark.connect.AnalyzePlanResponse.unpersist:type_name -> spark.connect.AnalyzePlanResponse.Unpersist\n\t53,  // 30: spark.connect.AnalyzePlanResponse.get_storage_level:type_name -> spark.connect.AnalyzePlanResponse.GetStorageLevel\n\t54,  // 31: spark.connect.AnalyzePlanResponse.json_to_ddl:type_name -> spark.connect.AnalyzePlanResponse.JsonToDDL\n\t4,   // 32: spark.connect.ExecutePlanRequest.user_context:type_name -> spark.connect.UserContext\n\t3,   // 33: spark.connect.ExecutePlanRequest.plan:type_name -> spark.connect.Plan\n\t55,  // 34: spark.connect.ExecutePlanRequest.request_options:type_name -> spark.connect.ExecutePlanRequest.RequestOption\n\t57,  // 35: spark.connect.ExecutePlanResponse.arrow_batch:type_name -> spark.connect.ExecutePlanResponse.ArrowBatch\n\t56,  // 36: spark.connect.ExecutePlanResponse.sql_command_result:type_name -> spark.connect.ExecutePlanResponse.SqlCommandResult\n\t91,  // 37: spark.connect.ExecutePlanResponse.write_stream_operation_start_result:type_name -> spark.connect.WriteStreamOperationStartResult\n\t92,  // 38: spark.connect.ExecutePlanResponse.streaming_query_command_result:type_name -> spark.connect.StreamingQueryCommandResult\n\t93,  // 39: spark.connect.ExecutePlanResponse.get_resources_command_result:type_name -> spark.connect.GetResourcesCommandResult\n\t94,  // 40: spark.connect.ExecutePlanResponse.streaming_query_manager_command_result:type_name -> spark.connect.StreamingQueryManagerCommandResult\n\t95,  // 41: spark.connect.ExecutePlanResponse.streaming_query_listener_events_result:type_name -> spark.connect.StreamingQueryListenerEventsResult\n\t60,  // 42: spark.connect.ExecutePlanResponse.result_complete:type_name -> spark.connect.ExecutePlanResponse.ResultComplete\n\t96,  // 43: spark.connect.ExecutePlanResponse.create_resource_profile_command_result:type_name -> spark.connect.CreateResourceProfileCommandResult\n\t61,  // 44: spark.connect.ExecutePlanResponse.execution_progress:type_name -> spark.connect.ExecutePlanResponse.ExecutionProgress\n\t26,  // 45: spark.connect.ExecutePlanResponse.checkpoint_command_result:type_name -> spark.connect.CheckpointCommandResult\n\t97,  // 46: spark.connect.ExecutePlanResponse.ml_command_result:type_name -> spark.connect.MlCommandResult\n\t98,  // 47: spark.connect.ExecutePlanResponse.pipeline_event_result:type_name -> spark.connect.PipelineEventResult\n\t99,  // 48: spark.connect.ExecutePlanResponse.pipeline_command_result:type_name -> spark.connect.PipelineCommandResult\n\t90,  // 49: spark.connect.ExecutePlanResponse.extension:type_name -> google.protobuf.Any\n\t58,  // 50: spark.connect.ExecutePlanResponse.metrics:type_name -> spark.connect.ExecutePlanResponse.Metrics\n\t59,  // 51: spark.connect.ExecutePlanResponse.observed_metrics:type_name -> spark.connect.ExecutePlanResponse.ObservedMetrics\n\t100, // 52: spark.connect.ExecutePlanResponse.schema:type_name -> spark.connect.DataType\n\t4,   // 53: spark.connect.ConfigRequest.user_context:type_name -> spark.connect.UserContext\n\t66,  // 54: spark.connect.ConfigRequest.operation:type_name -> spark.connect.ConfigRequest.Operation\n\t9,   // 55: spark.connect.ConfigResponse.pairs:type_name -> spark.connect.KeyValue\n\t4,   // 56: spark.connect.AddArtifactsRequest.user_context:type_name -> spark.connect.UserContext\n\t76,  // 57: spark.connect.AddArtifactsRequest.batch:type_name -> spark.connect.AddArtifactsRequest.Batch\n\t77,  // 58: spark.connect.AddArtifactsRequest.begin_chunk:type_name -> spark.connect.AddArtifactsRequest.BeginChunkedArtifact\n\t74,  // 59: spark.connect.AddArtifactsRequest.chunk:type_name -> spark.connect.AddArtifactsRequest.ArtifactChunk\n\t78,  // 60: spark.connect.AddArtifactsResponse.artifacts:type_name -> spark.connect.AddArtifactsResponse.ArtifactSummary\n\t4,   // 61: spark.connect.ArtifactStatusesRequest.user_context:type_name -> spark.connect.UserContext\n\t79,  // 62: spark.connect.ArtifactStatusesResponse.statuses:type_name -> spark.connect.ArtifactStatusesResponse.StatusesEntry\n\t4,   // 63: spark.connect.InterruptRequest.user_context:type_name -> spark.connect.UserContext\n\t1,   // 64: spark.connect.InterruptRequest.interrupt_type:type_name -> spark.connect.InterruptRequest.InterruptType\n\t4,   // 65: spark.connect.ReattachExecuteRequest.user_context:type_name -> spark.connect.UserContext\n\t4,   // 66: spark.connect.ReleaseExecuteRequest.user_context:type_name -> spark.connect.UserContext\n\t81,  // 67: spark.connect.ReleaseExecuteRequest.release_all:type_name -> spark.connect.ReleaseExecuteRequest.ReleaseAll\n\t82,  // 68: spark.connect.ReleaseExecuteRequest.release_until:type_name -> spark.connect.ReleaseExecuteRequest.ReleaseUntil\n\t4,   // 69: spark.connect.ReleaseSessionRequest.user_context:type_name -> spark.connect.UserContext\n\t4,   // 70: spark.connect.FetchErrorDetailsRequest.user_context:type_name -> spark.connect.UserContext\n\t86,  // 71: spark.connect.FetchErrorDetailsResponse.errors:type_name -> spark.connect.FetchErrorDetailsResponse.Error\n\t101, // 72: spark.connect.CheckpointCommandResult.relation:type_name -> spark.connect.CachedRemoteRelation\n\t3,   // 73: spark.connect.AnalyzePlanRequest.Schema.plan:type_name -> spark.connect.Plan\n\t3,   // 74: spark.connect.AnalyzePlanRequest.Explain.plan:type_name -> spark.connect.Plan\n\t0,   // 75: spark.connect.AnalyzePlanRequest.Explain.explain_mode:type_name -> spark.connect.AnalyzePlanRequest.Explain.ExplainMode\n\t3,   // 76: spark.connect.AnalyzePlanRequest.TreeString.plan:type_name -> spark.connect.Plan\n\t3,   // 77: spark.connect.AnalyzePlanRequest.IsLocal.plan:type_name -> spark.connect.Plan\n\t3,   // 78: spark.connect.AnalyzePlanRequest.IsStreaming.plan:type_name -> spark.connect.Plan\n\t3,   // 79: spark.connect.AnalyzePlanRequest.InputFiles.plan:type_name -> spark.connect.Plan\n\t3,   // 80: spark.connect.AnalyzePlanRequest.SameSemantics.target_plan:type_name -> spark.connect.Plan\n\t3,   // 81: spark.connect.AnalyzePlanRequest.SameSemantics.other_plan:type_name -> spark.connect.Plan\n\t3,   // 82: spark.connect.AnalyzePlanRequest.SemanticHash.plan:type_name -> spark.connect.Plan\n\t88,  // 83: spark.connect.AnalyzePlanRequest.Persist.relation:type_name -> spark.connect.Relation\n\t102, // 84: spark.connect.AnalyzePlanRequest.Persist.storage_level:type_name -> spark.connect.StorageLevel\n\t88,  // 85: spark.connect.AnalyzePlanRequest.Unpersist.relation:type_name -> spark.connect.Relation\n\t88,  // 86: spark.connect.AnalyzePlanRequest.GetStorageLevel.relation:type_name -> spark.connect.Relation\n\t100, // 87: spark.connect.AnalyzePlanResponse.Schema.schema:type_name -> spark.connect.DataType\n\t100, // 88: spark.connect.AnalyzePlanResponse.DDLParse.parsed:type_name -> spark.connect.DataType\n\t102, // 89: spark.connect.AnalyzePlanResponse.GetStorageLevel.storage_level:type_name -> spark.connect.StorageLevel\n\t18,  // 90: spark.connect.ExecutePlanRequest.RequestOption.reattach_options:type_name -> spark.connect.ReattachOptions\n\t90,  // 91: spark.connect.ExecutePlanRequest.RequestOption.extension:type_name -> google.protobuf.Any\n\t88,  // 92: spark.connect.ExecutePlanResponse.SqlCommandResult.relation:type_name -> spark.connect.Relation\n\t62,  // 93: spark.connect.ExecutePlanResponse.Metrics.metrics:type_name -> spark.connect.ExecutePlanResponse.Metrics.MetricObject\n\t103, // 94: spark.connect.ExecutePlanResponse.ObservedMetrics.values:type_name -> spark.connect.Expression.Literal\n\t65,  // 95: spark.connect.ExecutePlanResponse.ExecutionProgress.stages:type_name -> spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfo\n\t64,  // 96: spark.connect.ExecutePlanResponse.Metrics.MetricObject.execution_metrics:type_name -> spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry\n\t63,  // 97: spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry.value:type_name -> spark.connect.ExecutePlanResponse.Metrics.MetricValue\n\t67,  // 98: spark.connect.ConfigRequest.Operation.set:type_name -> spark.connect.ConfigRequest.Set\n\t68,  // 99: spark.connect.ConfigRequest.Operation.get:type_name -> spark.connect.ConfigRequest.Get\n\t69,  // 100: spark.connect.ConfigRequest.Operation.get_with_default:type_name -> spark.connect.ConfigRequest.GetWithDefault\n\t70,  // 101: spark.connect.ConfigRequest.Operation.get_option:type_name -> spark.connect.ConfigRequest.GetOption\n\t71,  // 102: spark.connect.ConfigRequest.Operation.get_all:type_name -> spark.connect.ConfigRequest.GetAll\n\t72,  // 103: spark.connect.ConfigRequest.Operation.unset:type_name -> spark.connect.ConfigRequest.Unset\n\t73,  // 104: spark.connect.ConfigRequest.Operation.is_modifiable:type_name -> spark.connect.ConfigRequest.IsModifiable\n\t9,   // 105: spark.connect.ConfigRequest.Set.pairs:type_name -> spark.connect.KeyValue\n\t9,   // 106: spark.connect.ConfigRequest.GetWithDefault.pairs:type_name -> spark.connect.KeyValue\n\t74,  // 107: spark.connect.AddArtifactsRequest.SingleChunkArtifact.data:type_name -> spark.connect.AddArtifactsRequest.ArtifactChunk\n\t75,  // 108: spark.connect.AddArtifactsRequest.Batch.artifacts:type_name -> spark.connect.AddArtifactsRequest.SingleChunkArtifact\n\t74,  // 109: spark.connect.AddArtifactsRequest.BeginChunkedArtifact.initial_chunk:type_name -> spark.connect.AddArtifactsRequest.ArtifactChunk\n\t80,  // 110: spark.connect.ArtifactStatusesResponse.StatusesEntry.value:type_name -> spark.connect.ArtifactStatusesResponse.ArtifactStatus\n\t2,   // 111: spark.connect.FetchErrorDetailsResponse.QueryContext.context_type:type_name -> spark.connect.FetchErrorDetailsResponse.QueryContext.ContextType\n\t87,  // 112: spark.connect.FetchErrorDetailsResponse.SparkThrowable.message_parameters:type_name -> spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntry\n\t84,  // 113: spark.connect.FetchErrorDetailsResponse.SparkThrowable.query_contexts:type_name -> spark.connect.FetchErrorDetailsResponse.QueryContext\n\t83,  // 114: spark.connect.FetchErrorDetailsResponse.Error.stack_trace:type_name -> spark.connect.FetchErrorDetailsResponse.StackTraceElement\n\t85,  // 115: spark.connect.FetchErrorDetailsResponse.Error.spark_throwable:type_name -> spark.connect.FetchErrorDetailsResponse.SparkThrowable\n\t7,   // 116: spark.connect.SparkConnectService.ExecutePlan:input_type -> spark.connect.ExecutePlanRequest\n\t5,   // 117: spark.connect.SparkConnectService.AnalyzePlan:input_type -> spark.connect.AnalyzePlanRequest\n\t10,  // 118: spark.connect.SparkConnectService.Config:input_type -> spark.connect.ConfigRequest\n\t12,  // 119: spark.connect.SparkConnectService.AddArtifacts:input_type -> spark.connect.AddArtifactsRequest\n\t14,  // 120: spark.connect.SparkConnectService.ArtifactStatus:input_type -> spark.connect.ArtifactStatusesRequest\n\t16,  // 121: spark.connect.SparkConnectService.Interrupt:input_type -> spark.connect.InterruptRequest\n\t19,  // 122: spark.connect.SparkConnectService.ReattachExecute:input_type -> spark.connect.ReattachExecuteRequest\n\t20,  // 123: spark.connect.SparkConnectService.ReleaseExecute:input_type -> spark.connect.ReleaseExecuteRequest\n\t22,  // 124: spark.connect.SparkConnectService.ReleaseSession:input_type -> spark.connect.ReleaseSessionRequest\n\t24,  // 125: spark.connect.SparkConnectService.FetchErrorDetails:input_type -> spark.connect.FetchErrorDetailsRequest\n\t8,   // 126: spark.connect.SparkConnectService.ExecutePlan:output_type -> spark.connect.ExecutePlanResponse\n\t6,   // 127: spark.connect.SparkConnectService.AnalyzePlan:output_type -> spark.connect.AnalyzePlanResponse\n\t11,  // 128: spark.connect.SparkConnectService.Config:output_type -> spark.connect.ConfigResponse\n\t13,  // 129: spark.connect.SparkConnectService.AddArtifacts:output_type -> spark.connect.AddArtifactsResponse\n\t15,  // 130: spark.connect.SparkConnectService.ArtifactStatus:output_type -> spark.connect.ArtifactStatusesResponse\n\t17,  // 131: spark.connect.SparkConnectService.Interrupt:output_type -> spark.connect.InterruptResponse\n\t8,   // 132: spark.connect.SparkConnectService.ReattachExecute:output_type -> spark.connect.ExecutePlanResponse\n\t21,  // 133: spark.connect.SparkConnectService.ReleaseExecute:output_type -> spark.connect.ReleaseExecuteResponse\n\t23,  // 134: spark.connect.SparkConnectService.ReleaseSession:output_type -> spark.connect.ReleaseSessionResponse\n\t25,  // 135: spark.connect.SparkConnectService.FetchErrorDetails:output_type -> spark.connect.FetchErrorDetailsResponse\n\t126, // [126:136] is the sub-list for method output_type\n\t116, // [116:126] is the sub-list for method input_type\n\t116, // [116:116] is the sub-list for extension type_name\n\t116, // [116:116] is the sub-list for extension extendee\n\t0,   // [0:116] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_base_proto_init() }\nfunc file_spark_connect_base_proto_init() {\n\tif File_spark_connect_base_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_commands_proto_init()\n\tfile_spark_connect_common_proto_init()\n\tfile_spark_connect_expressions_proto_init()\n\tfile_spark_connect_relations_proto_init()\n\tfile_spark_connect_types_proto_init()\n\tfile_spark_connect_ml_proto_init()\n\tfile_spark_connect_pipelines_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_base_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Plan); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*UserContext); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*KeyValue); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ArtifactStatusesRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ArtifactStatusesResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*InterruptRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*InterruptResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReattachOptions); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReattachExecuteRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseExecuteRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseExecuteResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseSessionRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseSessionResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsResponse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CheckpointCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_Schema); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_Explain); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_TreeString); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_IsLocal); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_IsStreaming); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_InputFiles); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_SparkVersion); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_DDLParse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_SameSemantics); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_SemanticHash); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_Persist); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_Unpersist); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_GetStorageLevel); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanRequest_JsonToDDL); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_Schema); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_Explain); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_TreeString); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_IsLocal); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_IsStreaming); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_InputFiles); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_SparkVersion); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_DDLParse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_SameSemantics); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_SemanticHash); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_Persist); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_Unpersist); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_GetStorageLevel); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AnalyzePlanResponse_JsonToDDL); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanRequest_RequestOption); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_SqlCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_ArrowBatch); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_Metrics); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_ObservedMetrics); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_ResultComplete); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_ExecutionProgress); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_Metrics_MetricObject); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_Metrics_MetricValue); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutePlanResponse_ExecutionProgress_StageInfo); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_Operation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_Set); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_Get); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_GetWithDefault); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_GetOption); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_GetAll); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_Unset); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ConfigRequest_IsModifiable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsRequest_ArtifactChunk); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsRequest_SingleChunkArtifact); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsRequest_Batch); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsRequest_BeginChunkedArtifact); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AddArtifactsResponse_ArtifactSummary); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ArtifactStatusesResponse_ArtifactStatus); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseExecuteRequest_ReleaseAll); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ReleaseExecuteRequest_ReleaseUntil); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsResponse_StackTraceElement); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsResponse_QueryContext); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsResponse_SparkThrowable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_base_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FetchErrorDetailsResponse_Error); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_base_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*Plan_Root)(nil),\n\t\t(*Plan_Command)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[2].OneofWrappers = []interface{}{\n\t\t(*AnalyzePlanRequest_Schema_)(nil),\n\t\t(*AnalyzePlanRequest_Explain_)(nil),\n\t\t(*AnalyzePlanRequest_TreeString_)(nil),\n\t\t(*AnalyzePlanRequest_IsLocal_)(nil),\n\t\t(*AnalyzePlanRequest_IsStreaming_)(nil),\n\t\t(*AnalyzePlanRequest_InputFiles_)(nil),\n\t\t(*AnalyzePlanRequest_SparkVersion_)(nil),\n\t\t(*AnalyzePlanRequest_DdlParse)(nil),\n\t\t(*AnalyzePlanRequest_SameSemantics_)(nil),\n\t\t(*AnalyzePlanRequest_SemanticHash_)(nil),\n\t\t(*AnalyzePlanRequest_Persist_)(nil),\n\t\t(*AnalyzePlanRequest_Unpersist_)(nil),\n\t\t(*AnalyzePlanRequest_GetStorageLevel_)(nil),\n\t\t(*AnalyzePlanRequest_JsonToDdl)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[3].OneofWrappers = []interface{}{\n\t\t(*AnalyzePlanResponse_Schema_)(nil),\n\t\t(*AnalyzePlanResponse_Explain_)(nil),\n\t\t(*AnalyzePlanResponse_TreeString_)(nil),\n\t\t(*AnalyzePlanResponse_IsLocal_)(nil),\n\t\t(*AnalyzePlanResponse_IsStreaming_)(nil),\n\t\t(*AnalyzePlanResponse_InputFiles_)(nil),\n\t\t(*AnalyzePlanResponse_SparkVersion_)(nil),\n\t\t(*AnalyzePlanResponse_DdlParse)(nil),\n\t\t(*AnalyzePlanResponse_SameSemantics_)(nil),\n\t\t(*AnalyzePlanResponse_SemanticHash_)(nil),\n\t\t(*AnalyzePlanResponse_Persist_)(nil),\n\t\t(*AnalyzePlanResponse_Unpersist_)(nil),\n\t\t(*AnalyzePlanResponse_GetStorageLevel_)(nil),\n\t\t(*AnalyzePlanResponse_JsonToDdl)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[4].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[5].OneofWrappers = []interface{}{\n\t\t(*ExecutePlanResponse_ArrowBatch_)(nil),\n\t\t(*ExecutePlanResponse_SqlCommandResult_)(nil),\n\t\t(*ExecutePlanResponse_WriteStreamOperationStartResult)(nil),\n\t\t(*ExecutePlanResponse_StreamingQueryCommandResult)(nil),\n\t\t(*ExecutePlanResponse_GetResourcesCommandResult)(nil),\n\t\t(*ExecutePlanResponse_StreamingQueryManagerCommandResult)(nil),\n\t\t(*ExecutePlanResponse_StreamingQueryListenerEventsResult)(nil),\n\t\t(*ExecutePlanResponse_ResultComplete_)(nil),\n\t\t(*ExecutePlanResponse_CreateResourceProfileCommandResult)(nil),\n\t\t(*ExecutePlanResponse_ExecutionProgress_)(nil),\n\t\t(*ExecutePlanResponse_CheckpointCommandResult)(nil),\n\t\t(*ExecutePlanResponse_MlCommandResult)(nil),\n\t\t(*ExecutePlanResponse_PipelineEventResult)(nil),\n\t\t(*ExecutePlanResponse_PipelineCommandResult)(nil),\n\t\t(*ExecutePlanResponse_Extension)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[6].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[7].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[9].OneofWrappers = []interface{}{\n\t\t(*AddArtifactsRequest_Batch_)(nil),\n\t\t(*AddArtifactsRequest_BeginChunk)(nil),\n\t\t(*AddArtifactsRequest_Chunk)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[11].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[13].OneofWrappers = []interface{}{\n\t\t(*InterruptRequest_OperationTag)(nil),\n\t\t(*InterruptRequest_OperationId)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[16].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[17].OneofWrappers = []interface{}{\n\t\t(*ReleaseExecuteRequest_ReleaseAll_)(nil),\n\t\t(*ReleaseExecuteRequest_ReleaseUntil_)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[18].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[19].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[21].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[22].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[26].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[34].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[35].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[52].OneofWrappers = []interface{}{\n\t\t(*ExecutePlanRequest_RequestOption_ReattachOptions)(nil),\n\t\t(*ExecutePlanRequest_RequestOption_Extension)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[54].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[63].OneofWrappers = []interface{}{\n\t\t(*ConfigRequest_Operation_Set)(nil),\n\t\t(*ConfigRequest_Operation_Get)(nil),\n\t\t(*ConfigRequest_Operation_GetWithDefault)(nil),\n\t\t(*ConfigRequest_Operation_GetOption)(nil),\n\t\t(*ConfigRequest_Operation_GetAll)(nil),\n\t\t(*ConfigRequest_Operation_Unset)(nil),\n\t\t(*ConfigRequest_Operation_IsModifiable)(nil),\n\t}\n\tfile_spark_connect_base_proto_msgTypes[64].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[68].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[80].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[82].OneofWrappers = []interface{}{}\n\tfile_spark_connect_base_proto_msgTypes[83].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_base_proto_rawDesc,\n\t\t\tNumEnums:      3,\n\t\t\tNumMessages:   85,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   1,\n\t\t},\n\t\tGoTypes:           file_spark_connect_base_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_base_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_base_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_base_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_base_proto = out.File\n\tfile_spark_connect_base_proto_rawDesc = nil\n\tfile_spark_connect_base_proto_goTypes = nil\n\tfile_spark_connect_base_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/base_grpc.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go-grpc. DO NOT EDIT.\n// versions:\n// - protoc-gen-go-grpc v1.3.0\n// - protoc             (unknown)\n// source: spark/connect/base.proto\n\npackage generated\n\nimport (\n\tcontext \"context\"\n\tgrpc \"google.golang.org/grpc\"\n\tcodes \"google.golang.org/grpc/codes\"\n\tstatus \"google.golang.org/grpc/status\"\n)\n\n// This is a compile-time assertion to ensure that this generated file\n// is compatible with the grpc package it is being compiled against.\n// Requires gRPC-Go v1.32.0 or later.\nconst _ = grpc.SupportPackageIsVersion7\n\nconst (\n\tSparkConnectService_ExecutePlan_FullMethodName       = \"/spark.connect.SparkConnectService/ExecutePlan\"\n\tSparkConnectService_AnalyzePlan_FullMethodName       = \"/spark.connect.SparkConnectService/AnalyzePlan\"\n\tSparkConnectService_Config_FullMethodName            = \"/spark.connect.SparkConnectService/Config\"\n\tSparkConnectService_AddArtifacts_FullMethodName      = \"/spark.connect.SparkConnectService/AddArtifacts\"\n\tSparkConnectService_ArtifactStatus_FullMethodName    = \"/spark.connect.SparkConnectService/ArtifactStatus\"\n\tSparkConnectService_Interrupt_FullMethodName         = \"/spark.connect.SparkConnectService/Interrupt\"\n\tSparkConnectService_ReattachExecute_FullMethodName   = \"/spark.connect.SparkConnectService/ReattachExecute\"\n\tSparkConnectService_ReleaseExecute_FullMethodName    = \"/spark.connect.SparkConnectService/ReleaseExecute\"\n\tSparkConnectService_ReleaseSession_FullMethodName    = \"/spark.connect.SparkConnectService/ReleaseSession\"\n\tSparkConnectService_FetchErrorDetails_FullMethodName = \"/spark.connect.SparkConnectService/FetchErrorDetails\"\n)\n\n// SparkConnectServiceClient is the client API for SparkConnectService service.\n//\n// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.\ntype SparkConnectServiceClient interface {\n\t// Executes a request that contains the query and returns a stream of [[Response]].\n\t//\n\t// It is guaranteed that there is at least one ARROW batch returned even if the result set is empty.\n\tExecutePlan(ctx context.Context, in *ExecutePlanRequest, opts ...grpc.CallOption) (SparkConnectService_ExecutePlanClient, error)\n\t// Analyzes a query and returns a [[AnalyzeResponse]] containing metadata about the query.\n\tAnalyzePlan(ctx context.Context, in *AnalyzePlanRequest, opts ...grpc.CallOption) (*AnalyzePlanResponse, error)\n\t// Update or fetch the configurations and returns a [[ConfigResponse]] containing the result.\n\tConfig(ctx context.Context, in *ConfigRequest, opts ...grpc.CallOption) (*ConfigResponse, error)\n\t// Add artifacts to the session and returns a [[AddArtifactsResponse]] containing metadata about\n\t// the added artifacts.\n\tAddArtifacts(ctx context.Context, opts ...grpc.CallOption) (SparkConnectService_AddArtifactsClient, error)\n\t// Check statuses of artifacts in the session and returns them in a [[ArtifactStatusesResponse]]\n\tArtifactStatus(ctx context.Context, in *ArtifactStatusesRequest, opts ...grpc.CallOption) (*ArtifactStatusesResponse, error)\n\t// Interrupts running executions\n\tInterrupt(ctx context.Context, in *InterruptRequest, opts ...grpc.CallOption) (*InterruptResponse, error)\n\t// Reattach to an existing reattachable execution.\n\t// The ExecutePlan must have been started with ReattachOptions.reattachable=true.\n\t// If the ExecutePlanResponse stream ends without a ResultComplete message, there is more to\n\t// continue. If there is a ResultComplete, the client should use ReleaseExecute with\n\tReattachExecute(ctx context.Context, in *ReattachExecuteRequest, opts ...grpc.CallOption) (SparkConnectService_ReattachExecuteClient, error)\n\t// Release an reattachable execution, or parts thereof.\n\t// The ExecutePlan must have been started with ReattachOptions.reattachable=true.\n\t// Non reattachable executions are released automatically and immediately after the ExecutePlan\n\t// RPC and ReleaseExecute may not be used.\n\tReleaseExecute(ctx context.Context, in *ReleaseExecuteRequest, opts ...grpc.CallOption) (*ReleaseExecuteResponse, error)\n\t// Release a session.\n\t// All the executions in the session will be released. Any further requests for the session with\n\t// that session_id for the given user_id will fail. If the session didn't exist or was already\n\t// released, this is a noop.\n\tReleaseSession(ctx context.Context, in *ReleaseSessionRequest, opts ...grpc.CallOption) (*ReleaseSessionResponse, error)\n\t// FetchErrorDetails retrieves the matched exception with details based on a provided error id.\n\tFetchErrorDetails(ctx context.Context, in *FetchErrorDetailsRequest, opts ...grpc.CallOption) (*FetchErrorDetailsResponse, error)\n}\n\ntype sparkConnectServiceClient struct {\n\tcc grpc.ClientConnInterface\n}\n\nfunc NewSparkConnectServiceClient(cc grpc.ClientConnInterface) SparkConnectServiceClient {\n\treturn &sparkConnectServiceClient{cc}\n}\n\nfunc (c *sparkConnectServiceClient) ExecutePlan(ctx context.Context, in *ExecutePlanRequest, opts ...grpc.CallOption) (SparkConnectService_ExecutePlanClient, error) {\n\tstream, err := c.cc.NewStream(ctx, &SparkConnectService_ServiceDesc.Streams[0], SparkConnectService_ExecutePlan_FullMethodName, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tx := &sparkConnectServiceExecutePlanClient{stream}\n\tif err := x.ClientStream.SendMsg(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif err := x.ClientStream.CloseSend(); err != nil {\n\t\treturn nil, err\n\t}\n\treturn x, nil\n}\n\ntype SparkConnectService_ExecutePlanClient interface {\n\tRecv() (*ExecutePlanResponse, error)\n\tgrpc.ClientStream\n}\n\ntype sparkConnectServiceExecutePlanClient struct {\n\tgrpc.ClientStream\n}\n\nfunc (x *sparkConnectServiceExecutePlanClient) Recv() (*ExecutePlanResponse, error) {\n\tm := new(ExecutePlanResponse)\n\tif err := x.ClientStream.RecvMsg(m); err != nil {\n\t\treturn nil, err\n\t}\n\treturn m, nil\n}\n\nfunc (c *sparkConnectServiceClient) AnalyzePlan(ctx context.Context, in *AnalyzePlanRequest, opts ...grpc.CallOption) (*AnalyzePlanResponse, error) {\n\tout := new(AnalyzePlanResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_AnalyzePlan_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) Config(ctx context.Context, in *ConfigRequest, opts ...grpc.CallOption) (*ConfigResponse, error) {\n\tout := new(ConfigResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_Config_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) AddArtifacts(ctx context.Context, opts ...grpc.CallOption) (SparkConnectService_AddArtifactsClient, error) {\n\tstream, err := c.cc.NewStream(ctx, &SparkConnectService_ServiceDesc.Streams[1], SparkConnectService_AddArtifacts_FullMethodName, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tx := &sparkConnectServiceAddArtifactsClient{stream}\n\treturn x, nil\n}\n\ntype SparkConnectService_AddArtifactsClient interface {\n\tSend(*AddArtifactsRequest) error\n\tCloseAndRecv() (*AddArtifactsResponse, error)\n\tgrpc.ClientStream\n}\n\ntype sparkConnectServiceAddArtifactsClient struct {\n\tgrpc.ClientStream\n}\n\nfunc (x *sparkConnectServiceAddArtifactsClient) Send(m *AddArtifactsRequest) error {\n\treturn x.ClientStream.SendMsg(m)\n}\n\nfunc (x *sparkConnectServiceAddArtifactsClient) CloseAndRecv() (*AddArtifactsResponse, error) {\n\tif err := x.ClientStream.CloseSend(); err != nil {\n\t\treturn nil, err\n\t}\n\tm := new(AddArtifactsResponse)\n\tif err := x.ClientStream.RecvMsg(m); err != nil {\n\t\treturn nil, err\n\t}\n\treturn m, nil\n}\n\nfunc (c *sparkConnectServiceClient) ArtifactStatus(ctx context.Context, in *ArtifactStatusesRequest, opts ...grpc.CallOption) (*ArtifactStatusesResponse, error) {\n\tout := new(ArtifactStatusesResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_ArtifactStatus_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) Interrupt(ctx context.Context, in *InterruptRequest, opts ...grpc.CallOption) (*InterruptResponse, error) {\n\tout := new(InterruptResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_Interrupt_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) ReattachExecute(ctx context.Context, in *ReattachExecuteRequest, opts ...grpc.CallOption) (SparkConnectService_ReattachExecuteClient, error) {\n\tstream, err := c.cc.NewStream(ctx, &SparkConnectService_ServiceDesc.Streams[2], SparkConnectService_ReattachExecute_FullMethodName, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tx := &sparkConnectServiceReattachExecuteClient{stream}\n\tif err := x.ClientStream.SendMsg(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif err := x.ClientStream.CloseSend(); err != nil {\n\t\treturn nil, err\n\t}\n\treturn x, nil\n}\n\ntype SparkConnectService_ReattachExecuteClient interface {\n\tRecv() (*ExecutePlanResponse, error)\n\tgrpc.ClientStream\n}\n\ntype sparkConnectServiceReattachExecuteClient struct {\n\tgrpc.ClientStream\n}\n\nfunc (x *sparkConnectServiceReattachExecuteClient) Recv() (*ExecutePlanResponse, error) {\n\tm := new(ExecutePlanResponse)\n\tif err := x.ClientStream.RecvMsg(m); err != nil {\n\t\treturn nil, err\n\t}\n\treturn m, nil\n}\n\nfunc (c *sparkConnectServiceClient) ReleaseExecute(ctx context.Context, in *ReleaseExecuteRequest, opts ...grpc.CallOption) (*ReleaseExecuteResponse, error) {\n\tout := new(ReleaseExecuteResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_ReleaseExecute_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) ReleaseSession(ctx context.Context, in *ReleaseSessionRequest, opts ...grpc.CallOption) (*ReleaseSessionResponse, error) {\n\tout := new(ReleaseSessionResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_ReleaseSession_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\nfunc (c *sparkConnectServiceClient) FetchErrorDetails(ctx context.Context, in *FetchErrorDetailsRequest, opts ...grpc.CallOption) (*FetchErrorDetailsResponse, error) {\n\tout := new(FetchErrorDetailsResponse)\n\terr := c.cc.Invoke(ctx, SparkConnectService_FetchErrorDetails_FullMethodName, in, out, opts...)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn out, nil\n}\n\n// SparkConnectServiceServer is the server API for SparkConnectService service.\n// All implementations must embed UnimplementedSparkConnectServiceServer\n// for forward compatibility\ntype SparkConnectServiceServer interface {\n\t// Executes a request that contains the query and returns a stream of [[Response]].\n\t//\n\t// It is guaranteed that there is at least one ARROW batch returned even if the result set is empty.\n\tExecutePlan(*ExecutePlanRequest, SparkConnectService_ExecutePlanServer) error\n\t// Analyzes a query and returns a [[AnalyzeResponse]] containing metadata about the query.\n\tAnalyzePlan(context.Context, *AnalyzePlanRequest) (*AnalyzePlanResponse, error)\n\t// Update or fetch the configurations and returns a [[ConfigResponse]] containing the result.\n\tConfig(context.Context, *ConfigRequest) (*ConfigResponse, error)\n\t// Add artifacts to the session and returns a [[AddArtifactsResponse]] containing metadata about\n\t// the added artifacts.\n\tAddArtifacts(SparkConnectService_AddArtifactsServer) error\n\t// Check statuses of artifacts in the session and returns them in a [[ArtifactStatusesResponse]]\n\tArtifactStatus(context.Context, *ArtifactStatusesRequest) (*ArtifactStatusesResponse, error)\n\t// Interrupts running executions\n\tInterrupt(context.Context, *InterruptRequest) (*InterruptResponse, error)\n\t// Reattach to an existing reattachable execution.\n\t// The ExecutePlan must have been started with ReattachOptions.reattachable=true.\n\t// If the ExecutePlanResponse stream ends without a ResultComplete message, there is more to\n\t// continue. If there is a ResultComplete, the client should use ReleaseExecute with\n\tReattachExecute(*ReattachExecuteRequest, SparkConnectService_ReattachExecuteServer) error\n\t// Release an reattachable execution, or parts thereof.\n\t// The ExecutePlan must have been started with ReattachOptions.reattachable=true.\n\t// Non reattachable executions are released automatically and immediately after the ExecutePlan\n\t// RPC and ReleaseExecute may not be used.\n\tReleaseExecute(context.Context, *ReleaseExecuteRequest) (*ReleaseExecuteResponse, error)\n\t// Release a session.\n\t// All the executions in the session will be released. Any further requests for the session with\n\t// that session_id for the given user_id will fail. If the session didn't exist or was already\n\t// released, this is a noop.\n\tReleaseSession(context.Context, *ReleaseSessionRequest) (*ReleaseSessionResponse, error)\n\t// FetchErrorDetails retrieves the matched exception with details based on a provided error id.\n\tFetchErrorDetails(context.Context, *FetchErrorDetailsRequest) (*FetchErrorDetailsResponse, error)\n\tmustEmbedUnimplementedSparkConnectServiceServer()\n}\n\n// UnimplementedSparkConnectServiceServer must be embedded to have forward compatible implementations.\ntype UnimplementedSparkConnectServiceServer struct {\n}\n\nfunc (UnimplementedSparkConnectServiceServer) ExecutePlan(*ExecutePlanRequest, SparkConnectService_ExecutePlanServer) error {\n\treturn status.Errorf(codes.Unimplemented, \"method ExecutePlan not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) AnalyzePlan(context.Context, *AnalyzePlanRequest) (*AnalyzePlanResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method AnalyzePlan not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) Config(context.Context, *ConfigRequest) (*ConfigResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method Config not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) AddArtifacts(SparkConnectService_AddArtifactsServer) error {\n\treturn status.Errorf(codes.Unimplemented, \"method AddArtifacts not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) ArtifactStatus(context.Context, *ArtifactStatusesRequest) (*ArtifactStatusesResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method ArtifactStatus not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) Interrupt(context.Context, *InterruptRequest) (*InterruptResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method Interrupt not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) ReattachExecute(*ReattachExecuteRequest, SparkConnectService_ReattachExecuteServer) error {\n\treturn status.Errorf(codes.Unimplemented, \"method ReattachExecute not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) ReleaseExecute(context.Context, *ReleaseExecuteRequest) (*ReleaseExecuteResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method ReleaseExecute not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) ReleaseSession(context.Context, *ReleaseSessionRequest) (*ReleaseSessionResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method ReleaseSession not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) FetchErrorDetails(context.Context, *FetchErrorDetailsRequest) (*FetchErrorDetailsResponse, error) {\n\treturn nil, status.Errorf(codes.Unimplemented, \"method FetchErrorDetails not implemented\")\n}\nfunc (UnimplementedSparkConnectServiceServer) mustEmbedUnimplementedSparkConnectServiceServer() {}\n\n// UnsafeSparkConnectServiceServer may be embedded to opt out of forward compatibility for this service.\n// Use of this interface is not recommended, as added methods to SparkConnectServiceServer will\n// result in compilation errors.\ntype UnsafeSparkConnectServiceServer interface {\n\tmustEmbedUnimplementedSparkConnectServiceServer()\n}\n\nfunc RegisterSparkConnectServiceServer(s grpc.ServiceRegistrar, srv SparkConnectServiceServer) {\n\ts.RegisterService(&SparkConnectService_ServiceDesc, srv)\n}\n\nfunc _SparkConnectService_ExecutePlan_Handler(srv interface{}, stream grpc.ServerStream) error {\n\tm := new(ExecutePlanRequest)\n\tif err := stream.RecvMsg(m); err != nil {\n\t\treturn err\n\t}\n\treturn srv.(SparkConnectServiceServer).ExecutePlan(m, &sparkConnectServiceExecutePlanServer{stream})\n}\n\ntype SparkConnectService_ExecutePlanServer interface {\n\tSend(*ExecutePlanResponse) error\n\tgrpc.ServerStream\n}\n\ntype sparkConnectServiceExecutePlanServer struct {\n\tgrpc.ServerStream\n}\n\nfunc (x *sparkConnectServiceExecutePlanServer) Send(m *ExecutePlanResponse) error {\n\treturn x.ServerStream.SendMsg(m)\n}\n\nfunc _SparkConnectService_AnalyzePlan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(AnalyzePlanRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).AnalyzePlan(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_AnalyzePlan_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).AnalyzePlan(ctx, req.(*AnalyzePlanRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_Config_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(ConfigRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).Config(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_Config_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).Config(ctx, req.(*ConfigRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_AddArtifacts_Handler(srv interface{}, stream grpc.ServerStream) error {\n\treturn srv.(SparkConnectServiceServer).AddArtifacts(&sparkConnectServiceAddArtifactsServer{stream})\n}\n\ntype SparkConnectService_AddArtifactsServer interface {\n\tSendAndClose(*AddArtifactsResponse) error\n\tRecv() (*AddArtifactsRequest, error)\n\tgrpc.ServerStream\n}\n\ntype sparkConnectServiceAddArtifactsServer struct {\n\tgrpc.ServerStream\n}\n\nfunc (x *sparkConnectServiceAddArtifactsServer) SendAndClose(m *AddArtifactsResponse) error {\n\treturn x.ServerStream.SendMsg(m)\n}\n\nfunc (x *sparkConnectServiceAddArtifactsServer) Recv() (*AddArtifactsRequest, error) {\n\tm := new(AddArtifactsRequest)\n\tif err := x.ServerStream.RecvMsg(m); err != nil {\n\t\treturn nil, err\n\t}\n\treturn m, nil\n}\n\nfunc _SparkConnectService_ArtifactStatus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(ArtifactStatusesRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).ArtifactStatus(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_ArtifactStatus_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).ArtifactStatus(ctx, req.(*ArtifactStatusesRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_Interrupt_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(InterruptRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).Interrupt(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_Interrupt_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).Interrupt(ctx, req.(*InterruptRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_ReattachExecute_Handler(srv interface{}, stream grpc.ServerStream) error {\n\tm := new(ReattachExecuteRequest)\n\tif err := stream.RecvMsg(m); err != nil {\n\t\treturn err\n\t}\n\treturn srv.(SparkConnectServiceServer).ReattachExecute(m, &sparkConnectServiceReattachExecuteServer{stream})\n}\n\ntype SparkConnectService_ReattachExecuteServer interface {\n\tSend(*ExecutePlanResponse) error\n\tgrpc.ServerStream\n}\n\ntype sparkConnectServiceReattachExecuteServer struct {\n\tgrpc.ServerStream\n}\n\nfunc (x *sparkConnectServiceReattachExecuteServer) Send(m *ExecutePlanResponse) error {\n\treturn x.ServerStream.SendMsg(m)\n}\n\nfunc _SparkConnectService_ReleaseExecute_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(ReleaseExecuteRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).ReleaseExecute(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_ReleaseExecute_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).ReleaseExecute(ctx, req.(*ReleaseExecuteRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_ReleaseSession_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(ReleaseSessionRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).ReleaseSession(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_ReleaseSession_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).ReleaseSession(ctx, req.(*ReleaseSessionRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\nfunc _SparkConnectService_FetchErrorDetails_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {\n\tin := new(FetchErrorDetailsRequest)\n\tif err := dec(in); err != nil {\n\t\treturn nil, err\n\t}\n\tif interceptor == nil {\n\t\treturn srv.(SparkConnectServiceServer).FetchErrorDetails(ctx, in)\n\t}\n\tinfo := &grpc.UnaryServerInfo{\n\t\tServer:     srv,\n\t\tFullMethod: SparkConnectService_FetchErrorDetails_FullMethodName,\n\t}\n\thandler := func(ctx context.Context, req interface{}) (interface{}, error) {\n\t\treturn srv.(SparkConnectServiceServer).FetchErrorDetails(ctx, req.(*FetchErrorDetailsRequest))\n\t}\n\treturn interceptor(ctx, in, info, handler)\n}\n\n// SparkConnectService_ServiceDesc is the grpc.ServiceDesc for SparkConnectService service.\n// It's only intended for direct use with grpc.RegisterService,\n// and not to be introspected or modified (even as a copy)\nvar SparkConnectService_ServiceDesc = grpc.ServiceDesc{\n\tServiceName: \"spark.connect.SparkConnectService\",\n\tHandlerType: (*SparkConnectServiceServer)(nil),\n\tMethods: []grpc.MethodDesc{\n\t\t{\n\t\t\tMethodName: \"AnalyzePlan\",\n\t\t\tHandler:    _SparkConnectService_AnalyzePlan_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"Config\",\n\t\t\tHandler:    _SparkConnectService_Config_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"ArtifactStatus\",\n\t\t\tHandler:    _SparkConnectService_ArtifactStatus_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"Interrupt\",\n\t\t\tHandler:    _SparkConnectService_Interrupt_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"ReleaseExecute\",\n\t\t\tHandler:    _SparkConnectService_ReleaseExecute_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"ReleaseSession\",\n\t\t\tHandler:    _SparkConnectService_ReleaseSession_Handler,\n\t\t},\n\t\t{\n\t\t\tMethodName: \"FetchErrorDetails\",\n\t\t\tHandler:    _SparkConnectService_FetchErrorDetails_Handler,\n\t\t},\n\t},\n\tStreams: []grpc.StreamDesc{\n\t\t{\n\t\t\tStreamName:    \"ExecutePlan\",\n\t\t\tHandler:       _SparkConnectService_ExecutePlan_Handler,\n\t\t\tServerStreams: true,\n\t\t},\n\t\t{\n\t\t\tStreamName:    \"AddArtifacts\",\n\t\t\tHandler:       _SparkConnectService_AddArtifacts_Handler,\n\t\t\tClientStreams: true,\n\t\t},\n\t\t{\n\t\t\tStreamName:    \"ReattachExecute\",\n\t\t\tHandler:       _SparkConnectService_ReattachExecute_Handler,\n\t\t\tServerStreams: true,\n\t\t},\n\t},\n\tMetadata: \"spark/connect/base.proto\",\n}\n"
  },
  {
    "path": "internal/generated/catalog.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/catalog.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// Catalog messages are marked as unstable.\ntype Catalog struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to CatType:\n\t//\n\t//\t*Catalog_CurrentDatabase\n\t//\t*Catalog_SetCurrentDatabase\n\t//\t*Catalog_ListDatabases\n\t//\t*Catalog_ListTables\n\t//\t*Catalog_ListFunctions\n\t//\t*Catalog_ListColumns\n\t//\t*Catalog_GetDatabase\n\t//\t*Catalog_GetTable\n\t//\t*Catalog_GetFunction\n\t//\t*Catalog_DatabaseExists\n\t//\t*Catalog_TableExists\n\t//\t*Catalog_FunctionExists\n\t//\t*Catalog_CreateExternalTable\n\t//\t*Catalog_CreateTable\n\t//\t*Catalog_DropTempView\n\t//\t*Catalog_DropGlobalTempView\n\t//\t*Catalog_RecoverPartitions\n\t//\t*Catalog_IsCached\n\t//\t*Catalog_CacheTable\n\t//\t*Catalog_UncacheTable\n\t//\t*Catalog_ClearCache\n\t//\t*Catalog_RefreshTable\n\t//\t*Catalog_RefreshByPath\n\t//\t*Catalog_CurrentCatalog\n\t//\t*Catalog_SetCurrentCatalog\n\t//\t*Catalog_ListCatalogs\n\tCatType isCatalog_CatType `protobuf_oneof:\"cat_type\"`\n}\n\nfunc (x *Catalog) Reset() {\n\t*x = Catalog{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Catalog) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Catalog) ProtoMessage() {}\n\nfunc (x *Catalog) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Catalog.ProtoReflect.Descriptor instead.\nfunc (*Catalog) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *Catalog) GetCatType() isCatalog_CatType {\n\tif m != nil {\n\t\treturn m.CatType\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetCurrentDatabase() *CurrentDatabase {\n\tif x, ok := x.GetCatType().(*Catalog_CurrentDatabase); ok {\n\t\treturn x.CurrentDatabase\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetSetCurrentDatabase() *SetCurrentDatabase {\n\tif x, ok := x.GetCatType().(*Catalog_SetCurrentDatabase); ok {\n\t\treturn x.SetCurrentDatabase\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetListDatabases() *ListDatabases {\n\tif x, ok := x.GetCatType().(*Catalog_ListDatabases); ok {\n\t\treturn x.ListDatabases\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetListTables() *ListTables {\n\tif x, ok := x.GetCatType().(*Catalog_ListTables); ok {\n\t\treturn x.ListTables\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetListFunctions() *ListFunctions {\n\tif x, ok := x.GetCatType().(*Catalog_ListFunctions); ok {\n\t\treturn x.ListFunctions\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetListColumns() *ListColumns {\n\tif x, ok := x.GetCatType().(*Catalog_ListColumns); ok {\n\t\treturn x.ListColumns\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetGetDatabase() *GetDatabase {\n\tif x, ok := x.GetCatType().(*Catalog_GetDatabase); ok {\n\t\treturn x.GetDatabase\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetGetTable() *GetTable {\n\tif x, ok := x.GetCatType().(*Catalog_GetTable); ok {\n\t\treturn x.GetTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetGetFunction() *GetFunction {\n\tif x, ok := x.GetCatType().(*Catalog_GetFunction); ok {\n\t\treturn x.GetFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetDatabaseExists() *DatabaseExists {\n\tif x, ok := x.GetCatType().(*Catalog_DatabaseExists); ok {\n\t\treturn x.DatabaseExists\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetTableExists() *TableExists {\n\tif x, ok := x.GetCatType().(*Catalog_TableExists); ok {\n\t\treturn x.TableExists\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetFunctionExists() *FunctionExists {\n\tif x, ok := x.GetCatType().(*Catalog_FunctionExists); ok {\n\t\treturn x.FunctionExists\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetCreateExternalTable() *CreateExternalTable {\n\tif x, ok := x.GetCatType().(*Catalog_CreateExternalTable); ok {\n\t\treturn x.CreateExternalTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetCreateTable() *CreateTable {\n\tif x, ok := x.GetCatType().(*Catalog_CreateTable); ok {\n\t\treturn x.CreateTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetDropTempView() *DropTempView {\n\tif x, ok := x.GetCatType().(*Catalog_DropTempView); ok {\n\t\treturn x.DropTempView\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetDropGlobalTempView() *DropGlobalTempView {\n\tif x, ok := x.GetCatType().(*Catalog_DropGlobalTempView); ok {\n\t\treturn x.DropGlobalTempView\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetRecoverPartitions() *RecoverPartitions {\n\tif x, ok := x.GetCatType().(*Catalog_RecoverPartitions); ok {\n\t\treturn x.RecoverPartitions\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetIsCached() *IsCached {\n\tif x, ok := x.GetCatType().(*Catalog_IsCached); ok {\n\t\treturn x.IsCached\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetCacheTable() *CacheTable {\n\tif x, ok := x.GetCatType().(*Catalog_CacheTable); ok {\n\t\treturn x.CacheTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetUncacheTable() *UncacheTable {\n\tif x, ok := x.GetCatType().(*Catalog_UncacheTable); ok {\n\t\treturn x.UncacheTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetClearCache() *ClearCache {\n\tif x, ok := x.GetCatType().(*Catalog_ClearCache); ok {\n\t\treturn x.ClearCache\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetRefreshTable() *RefreshTable {\n\tif x, ok := x.GetCatType().(*Catalog_RefreshTable); ok {\n\t\treturn x.RefreshTable\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetRefreshByPath() *RefreshByPath {\n\tif x, ok := x.GetCatType().(*Catalog_RefreshByPath); ok {\n\t\treturn x.RefreshByPath\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetCurrentCatalog() *CurrentCatalog {\n\tif x, ok := x.GetCatType().(*Catalog_CurrentCatalog); ok {\n\t\treturn x.CurrentCatalog\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetSetCurrentCatalog() *SetCurrentCatalog {\n\tif x, ok := x.GetCatType().(*Catalog_SetCurrentCatalog); ok {\n\t\treturn x.SetCurrentCatalog\n\t}\n\treturn nil\n}\n\nfunc (x *Catalog) GetListCatalogs() *ListCatalogs {\n\tif x, ok := x.GetCatType().(*Catalog_ListCatalogs); ok {\n\t\treturn x.ListCatalogs\n\t}\n\treturn nil\n}\n\ntype isCatalog_CatType interface {\n\tisCatalog_CatType()\n}\n\ntype Catalog_CurrentDatabase struct {\n\tCurrentDatabase *CurrentDatabase `protobuf:\"bytes,1,opt,name=current_database,json=currentDatabase,proto3,oneof\"`\n}\n\ntype Catalog_SetCurrentDatabase struct {\n\tSetCurrentDatabase *SetCurrentDatabase `protobuf:\"bytes,2,opt,name=set_current_database,json=setCurrentDatabase,proto3,oneof\"`\n}\n\ntype Catalog_ListDatabases struct {\n\tListDatabases *ListDatabases `protobuf:\"bytes,3,opt,name=list_databases,json=listDatabases,proto3,oneof\"`\n}\n\ntype Catalog_ListTables struct {\n\tListTables *ListTables `protobuf:\"bytes,4,opt,name=list_tables,json=listTables,proto3,oneof\"`\n}\n\ntype Catalog_ListFunctions struct {\n\tListFunctions *ListFunctions `protobuf:\"bytes,5,opt,name=list_functions,json=listFunctions,proto3,oneof\"`\n}\n\ntype Catalog_ListColumns struct {\n\tListColumns *ListColumns `protobuf:\"bytes,6,opt,name=list_columns,json=listColumns,proto3,oneof\"`\n}\n\ntype Catalog_GetDatabase struct {\n\tGetDatabase *GetDatabase `protobuf:\"bytes,7,opt,name=get_database,json=getDatabase,proto3,oneof\"`\n}\n\ntype Catalog_GetTable struct {\n\tGetTable *GetTable `protobuf:\"bytes,8,opt,name=get_table,json=getTable,proto3,oneof\"`\n}\n\ntype Catalog_GetFunction struct {\n\tGetFunction *GetFunction `protobuf:\"bytes,9,opt,name=get_function,json=getFunction,proto3,oneof\"`\n}\n\ntype Catalog_DatabaseExists struct {\n\tDatabaseExists *DatabaseExists `protobuf:\"bytes,10,opt,name=database_exists,json=databaseExists,proto3,oneof\"`\n}\n\ntype Catalog_TableExists struct {\n\tTableExists *TableExists `protobuf:\"bytes,11,opt,name=table_exists,json=tableExists,proto3,oneof\"`\n}\n\ntype Catalog_FunctionExists struct {\n\tFunctionExists *FunctionExists `protobuf:\"bytes,12,opt,name=function_exists,json=functionExists,proto3,oneof\"`\n}\n\ntype Catalog_CreateExternalTable struct {\n\tCreateExternalTable *CreateExternalTable `protobuf:\"bytes,13,opt,name=create_external_table,json=createExternalTable,proto3,oneof\"`\n}\n\ntype Catalog_CreateTable struct {\n\tCreateTable *CreateTable `protobuf:\"bytes,14,opt,name=create_table,json=createTable,proto3,oneof\"`\n}\n\ntype Catalog_DropTempView struct {\n\tDropTempView *DropTempView `protobuf:\"bytes,15,opt,name=drop_temp_view,json=dropTempView,proto3,oneof\"`\n}\n\ntype Catalog_DropGlobalTempView struct {\n\tDropGlobalTempView *DropGlobalTempView `protobuf:\"bytes,16,opt,name=drop_global_temp_view,json=dropGlobalTempView,proto3,oneof\"`\n}\n\ntype Catalog_RecoverPartitions struct {\n\tRecoverPartitions *RecoverPartitions `protobuf:\"bytes,17,opt,name=recover_partitions,json=recoverPartitions,proto3,oneof\"`\n}\n\ntype Catalog_IsCached struct {\n\tIsCached *IsCached `protobuf:\"bytes,18,opt,name=is_cached,json=isCached,proto3,oneof\"`\n}\n\ntype Catalog_CacheTable struct {\n\tCacheTable *CacheTable `protobuf:\"bytes,19,opt,name=cache_table,json=cacheTable,proto3,oneof\"`\n}\n\ntype Catalog_UncacheTable struct {\n\tUncacheTable *UncacheTable `protobuf:\"bytes,20,opt,name=uncache_table,json=uncacheTable,proto3,oneof\"`\n}\n\ntype Catalog_ClearCache struct {\n\tClearCache *ClearCache `protobuf:\"bytes,21,opt,name=clear_cache,json=clearCache,proto3,oneof\"`\n}\n\ntype Catalog_RefreshTable struct {\n\tRefreshTable *RefreshTable `protobuf:\"bytes,22,opt,name=refresh_table,json=refreshTable,proto3,oneof\"`\n}\n\ntype Catalog_RefreshByPath struct {\n\tRefreshByPath *RefreshByPath `protobuf:\"bytes,23,opt,name=refresh_by_path,json=refreshByPath,proto3,oneof\"`\n}\n\ntype Catalog_CurrentCatalog struct {\n\tCurrentCatalog *CurrentCatalog `protobuf:\"bytes,24,opt,name=current_catalog,json=currentCatalog,proto3,oneof\"`\n}\n\ntype Catalog_SetCurrentCatalog struct {\n\tSetCurrentCatalog *SetCurrentCatalog `protobuf:\"bytes,25,opt,name=set_current_catalog,json=setCurrentCatalog,proto3,oneof\"`\n}\n\ntype Catalog_ListCatalogs struct {\n\tListCatalogs *ListCatalogs `protobuf:\"bytes,26,opt,name=list_catalogs,json=listCatalogs,proto3,oneof\"`\n}\n\nfunc (*Catalog_CurrentDatabase) isCatalog_CatType() {}\n\nfunc (*Catalog_SetCurrentDatabase) isCatalog_CatType() {}\n\nfunc (*Catalog_ListDatabases) isCatalog_CatType() {}\n\nfunc (*Catalog_ListTables) isCatalog_CatType() {}\n\nfunc (*Catalog_ListFunctions) isCatalog_CatType() {}\n\nfunc (*Catalog_ListColumns) isCatalog_CatType() {}\n\nfunc (*Catalog_GetDatabase) isCatalog_CatType() {}\n\nfunc (*Catalog_GetTable) isCatalog_CatType() {}\n\nfunc (*Catalog_GetFunction) isCatalog_CatType() {}\n\nfunc (*Catalog_DatabaseExists) isCatalog_CatType() {}\n\nfunc (*Catalog_TableExists) isCatalog_CatType() {}\n\nfunc (*Catalog_FunctionExists) isCatalog_CatType() {}\n\nfunc (*Catalog_CreateExternalTable) isCatalog_CatType() {}\n\nfunc (*Catalog_CreateTable) isCatalog_CatType() {}\n\nfunc (*Catalog_DropTempView) isCatalog_CatType() {}\n\nfunc (*Catalog_DropGlobalTempView) isCatalog_CatType() {}\n\nfunc (*Catalog_RecoverPartitions) isCatalog_CatType() {}\n\nfunc (*Catalog_IsCached) isCatalog_CatType() {}\n\nfunc (*Catalog_CacheTable) isCatalog_CatType() {}\n\nfunc (*Catalog_UncacheTable) isCatalog_CatType() {}\n\nfunc (*Catalog_ClearCache) isCatalog_CatType() {}\n\nfunc (*Catalog_RefreshTable) isCatalog_CatType() {}\n\nfunc (*Catalog_RefreshByPath) isCatalog_CatType() {}\n\nfunc (*Catalog_CurrentCatalog) isCatalog_CatType() {}\n\nfunc (*Catalog_SetCurrentCatalog) isCatalog_CatType() {}\n\nfunc (*Catalog_ListCatalogs) isCatalog_CatType() {}\n\n// See `spark.catalog.currentDatabase`\ntype CurrentDatabase struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *CurrentDatabase) Reset() {\n\t*x = CurrentDatabase{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CurrentDatabase) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CurrentDatabase) ProtoMessage() {}\n\nfunc (x *CurrentDatabase) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CurrentDatabase.ProtoReflect.Descriptor instead.\nfunc (*CurrentDatabase) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{1}\n}\n\n// See `spark.catalog.setCurrentDatabase`\ntype SetCurrentDatabase struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tDbName string `protobuf:\"bytes,1,opt,name=db_name,json=dbName,proto3\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *SetCurrentDatabase) Reset() {\n\t*x = SetCurrentDatabase{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SetCurrentDatabase) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SetCurrentDatabase) ProtoMessage() {}\n\nfunc (x *SetCurrentDatabase) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SetCurrentDatabase.ProtoReflect.Descriptor instead.\nfunc (*SetCurrentDatabase) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *SetCurrentDatabase) GetDbName() string {\n\tif x != nil {\n\t\treturn x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.listDatabases`\ntype ListDatabases struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The pattern that the database name needs to match\n\tPattern *string `protobuf:\"bytes,1,opt,name=pattern,proto3,oneof\" json:\"pattern,omitempty\"`\n}\n\nfunc (x *ListDatabases) Reset() {\n\t*x = ListDatabases{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ListDatabases) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ListDatabases) ProtoMessage() {}\n\nfunc (x *ListDatabases) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ListDatabases.ProtoReflect.Descriptor instead.\nfunc (*ListDatabases) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *ListDatabases) GetPattern() string {\n\tif x != nil && x.Pattern != nil {\n\t\treturn *x.Pattern\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.listTables`\ntype ListTables struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,1,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n\t// (Optional) The pattern that the table name needs to match\n\tPattern *string `protobuf:\"bytes,2,opt,name=pattern,proto3,oneof\" json:\"pattern,omitempty\"`\n}\n\nfunc (x *ListTables) Reset() {\n\t*x = ListTables{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ListTables) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ListTables) ProtoMessage() {}\n\nfunc (x *ListTables) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ListTables.ProtoReflect.Descriptor instead.\nfunc (*ListTables) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *ListTables) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\nfunc (x *ListTables) GetPattern() string {\n\tif x != nil && x.Pattern != nil {\n\t\treturn *x.Pattern\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.listFunctions`\ntype ListFunctions struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,1,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n\t// (Optional) The pattern that the function name needs to match\n\tPattern *string `protobuf:\"bytes,2,opt,name=pattern,proto3,oneof\" json:\"pattern,omitempty\"`\n}\n\nfunc (x *ListFunctions) Reset() {\n\t*x = ListFunctions{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ListFunctions) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ListFunctions) ProtoMessage() {}\n\nfunc (x *ListFunctions) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ListFunctions.ProtoReflect.Descriptor instead.\nfunc (*ListFunctions) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (x *ListFunctions) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\nfunc (x *ListFunctions) GetPattern() string {\n\tif x != nil && x.Pattern != nil {\n\t\treturn *x.Pattern\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.listColumns`\ntype ListColumns struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,2,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *ListColumns) Reset() {\n\t*x = ListColumns{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ListColumns) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ListColumns) ProtoMessage() {}\n\nfunc (x *ListColumns) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ListColumns.ProtoReflect.Descriptor instead.\nfunc (*ListColumns) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (x *ListColumns) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *ListColumns) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.getDatabase`\ntype GetDatabase struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tDbName string `protobuf:\"bytes,1,opt,name=db_name,json=dbName,proto3\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *GetDatabase) Reset() {\n\t*x = GetDatabase{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GetDatabase) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GetDatabase) ProtoMessage() {}\n\nfunc (x *GetDatabase) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GetDatabase.ProtoReflect.Descriptor instead.\nfunc (*GetDatabase) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (x *GetDatabase) GetDbName() string {\n\tif x != nil {\n\t\treturn x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.getTable`\ntype GetTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,2,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *GetTable) Reset() {\n\t*x = GetTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GetTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GetTable) ProtoMessage() {}\n\nfunc (x *GetTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GetTable.ProtoReflect.Descriptor instead.\nfunc (*GetTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *GetTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *GetTable) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.getFunction`\ntype GetFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,2,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *GetFunction) Reset() {\n\t*x = GetFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GetFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GetFunction) ProtoMessage() {}\n\nfunc (x *GetFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GetFunction.ProtoReflect.Descriptor instead.\nfunc (*GetFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *GetFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *GetFunction) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.databaseExists`\ntype DatabaseExists struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tDbName string `protobuf:\"bytes,1,opt,name=db_name,json=dbName,proto3\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *DatabaseExists) Reset() {\n\t*x = DatabaseExists{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DatabaseExists) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DatabaseExists) ProtoMessage() {}\n\nfunc (x *DatabaseExists) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DatabaseExists.ProtoReflect.Descriptor instead.\nfunc (*DatabaseExists) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *DatabaseExists) GetDbName() string {\n\tif x != nil {\n\t\treturn x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.tableExists`\ntype TableExists struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,2,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *TableExists) Reset() {\n\t*x = TableExists{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *TableExists) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*TableExists) ProtoMessage() {}\n\nfunc (x *TableExists) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use TableExists.ProtoReflect.Descriptor instead.\nfunc (*TableExists) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{11}\n}\n\nfunc (x *TableExists) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *TableExists) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.functionExists`\ntype FunctionExists struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional)\n\tDbName *string `protobuf:\"bytes,2,opt,name=db_name,json=dbName,proto3,oneof\" json:\"db_name,omitempty\"`\n}\n\nfunc (x *FunctionExists) Reset() {\n\t*x = FunctionExists{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *FunctionExists) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*FunctionExists) ProtoMessage() {}\n\nfunc (x *FunctionExists) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use FunctionExists.ProtoReflect.Descriptor instead.\nfunc (*FunctionExists) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{12}\n}\n\nfunc (x *FunctionExists) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *FunctionExists) GetDbName() string {\n\tif x != nil && x.DbName != nil {\n\t\treturn *x.DbName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.createExternalTable`\ntype CreateExternalTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tPath *string `protobuf:\"bytes,2,opt,name=path,proto3,oneof\" json:\"path,omitempty\"`\n\t// (Optional)\n\tSource *string `protobuf:\"bytes,3,opt,name=source,proto3,oneof\" json:\"source,omitempty\"`\n\t// (Optional)\n\tSchema *DataType `protobuf:\"bytes,4,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n\t// Options could be empty for valid data source format.\n\t// The map key is case insensitive.\n\tOptions map[string]string `protobuf:\"bytes,5,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *CreateExternalTable) Reset() {\n\t*x = CreateExternalTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CreateExternalTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CreateExternalTable) ProtoMessage() {}\n\nfunc (x *CreateExternalTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CreateExternalTable.ProtoReflect.Descriptor instead.\nfunc (*CreateExternalTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{13}\n}\n\nfunc (x *CreateExternalTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateExternalTable) GetPath() string {\n\tif x != nil && x.Path != nil {\n\t\treturn *x.Path\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateExternalTable) GetSource() string {\n\tif x != nil && x.Source != nil {\n\t\treturn *x.Source\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateExternalTable) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *CreateExternalTable) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\n// See `spark.catalog.createTable`\ntype CreateTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tPath *string `protobuf:\"bytes,2,opt,name=path,proto3,oneof\" json:\"path,omitempty\"`\n\t// (Optional)\n\tSource *string `protobuf:\"bytes,3,opt,name=source,proto3,oneof\" json:\"source,omitempty\"`\n\t// (Optional)\n\tDescription *string `protobuf:\"bytes,4,opt,name=description,proto3,oneof\" json:\"description,omitempty\"`\n\t// (Optional)\n\tSchema *DataType `protobuf:\"bytes,5,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n\t// Options could be empty for valid data source format.\n\t// The map key is case insensitive.\n\tOptions map[string]string `protobuf:\"bytes,6,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *CreateTable) Reset() {\n\t*x = CreateTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CreateTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CreateTable) ProtoMessage() {}\n\nfunc (x *CreateTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CreateTable.ProtoReflect.Descriptor instead.\nfunc (*CreateTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{14}\n}\n\nfunc (x *CreateTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateTable) GetPath() string {\n\tif x != nil && x.Path != nil {\n\t\treturn *x.Path\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateTable) GetSource() string {\n\tif x != nil && x.Source != nil {\n\t\treturn *x.Source\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateTable) GetDescription() string {\n\tif x != nil && x.Description != nil {\n\t\treturn *x.Description\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateTable) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *CreateTable) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\n// See `spark.catalog.dropTempView`\ntype DropTempView struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tViewName string `protobuf:\"bytes,1,opt,name=view_name,json=viewName,proto3\" json:\"view_name,omitempty\"`\n}\n\nfunc (x *DropTempView) Reset() {\n\t*x = DropTempView{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DropTempView) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DropTempView) ProtoMessage() {}\n\nfunc (x *DropTempView) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DropTempView.ProtoReflect.Descriptor instead.\nfunc (*DropTempView) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{15}\n}\n\nfunc (x *DropTempView) GetViewName() string {\n\tif x != nil {\n\t\treturn x.ViewName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.dropGlobalTempView`\ntype DropGlobalTempView struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tViewName string `protobuf:\"bytes,1,opt,name=view_name,json=viewName,proto3\" json:\"view_name,omitempty\"`\n}\n\nfunc (x *DropGlobalTempView) Reset() {\n\t*x = DropGlobalTempView{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DropGlobalTempView) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DropGlobalTempView) ProtoMessage() {}\n\nfunc (x *DropGlobalTempView) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DropGlobalTempView.ProtoReflect.Descriptor instead.\nfunc (*DropGlobalTempView) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{16}\n}\n\nfunc (x *DropGlobalTempView) GetViewName() string {\n\tif x != nil {\n\t\treturn x.ViewName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.recoverPartitions`\ntype RecoverPartitions struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n}\n\nfunc (x *RecoverPartitions) Reset() {\n\t*x = RecoverPartitions{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RecoverPartitions) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RecoverPartitions) ProtoMessage() {}\n\nfunc (x *RecoverPartitions) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RecoverPartitions.ProtoReflect.Descriptor instead.\nfunc (*RecoverPartitions) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{17}\n}\n\nfunc (x *RecoverPartitions) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.isCached`\ntype IsCached struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n}\n\nfunc (x *IsCached) Reset() {\n\t*x = IsCached{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *IsCached) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*IsCached) ProtoMessage() {}\n\nfunc (x *IsCached) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use IsCached.ProtoReflect.Descriptor instead.\nfunc (*IsCached) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{18}\n}\n\nfunc (x *IsCached) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.cacheTable`\ntype CacheTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional)\n\tStorageLevel *StorageLevel `protobuf:\"bytes,2,opt,name=storage_level,json=storageLevel,proto3,oneof\" json:\"storage_level,omitempty\"`\n}\n\nfunc (x *CacheTable) Reset() {\n\t*x = CacheTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CacheTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CacheTable) ProtoMessage() {}\n\nfunc (x *CacheTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CacheTable.ProtoReflect.Descriptor instead.\nfunc (*CacheTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{19}\n}\n\nfunc (x *CacheTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CacheTable) GetStorageLevel() *StorageLevel {\n\tif x != nil {\n\t\treturn x.StorageLevel\n\t}\n\treturn nil\n}\n\n// See `spark.catalog.uncacheTable`\ntype UncacheTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n}\n\nfunc (x *UncacheTable) Reset() {\n\t*x = UncacheTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *UncacheTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*UncacheTable) ProtoMessage() {}\n\nfunc (x *UncacheTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use UncacheTable.ProtoReflect.Descriptor instead.\nfunc (*UncacheTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{20}\n}\n\nfunc (x *UncacheTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.clearCache`\ntype ClearCache struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *ClearCache) Reset() {\n\t*x = ClearCache{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ClearCache) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ClearCache) ProtoMessage() {}\n\nfunc (x *ClearCache) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ClearCache.ProtoReflect.Descriptor instead.\nfunc (*ClearCache) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{21}\n}\n\n// See `spark.catalog.refreshTable`\ntype RefreshTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n}\n\nfunc (x *RefreshTable) Reset() {\n\t*x = RefreshTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RefreshTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RefreshTable) ProtoMessage() {}\n\nfunc (x *RefreshTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RefreshTable.ProtoReflect.Descriptor instead.\nfunc (*RefreshTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{22}\n}\n\nfunc (x *RefreshTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.refreshByPath`\ntype RefreshByPath struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tPath string `protobuf:\"bytes,1,opt,name=path,proto3\" json:\"path,omitempty\"`\n}\n\nfunc (x *RefreshByPath) Reset() {\n\t*x = RefreshByPath{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RefreshByPath) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RefreshByPath) ProtoMessage() {}\n\nfunc (x *RefreshByPath) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RefreshByPath.ProtoReflect.Descriptor instead.\nfunc (*RefreshByPath) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{23}\n}\n\nfunc (x *RefreshByPath) GetPath() string {\n\tif x != nil {\n\t\treturn x.Path\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.currentCatalog`\ntype CurrentCatalog struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *CurrentCatalog) Reset() {\n\t*x = CurrentCatalog{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[24]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CurrentCatalog) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CurrentCatalog) ProtoMessage() {}\n\nfunc (x *CurrentCatalog) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[24]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CurrentCatalog.ProtoReflect.Descriptor instead.\nfunc (*CurrentCatalog) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{24}\n}\n\n// See `spark.catalog.setCurrentCatalog`\ntype SetCurrentCatalog struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required)\n\tCatalogName string `protobuf:\"bytes,1,opt,name=catalog_name,json=catalogName,proto3\" json:\"catalog_name,omitempty\"`\n}\n\nfunc (x *SetCurrentCatalog) Reset() {\n\t*x = SetCurrentCatalog{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[25]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SetCurrentCatalog) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SetCurrentCatalog) ProtoMessage() {}\n\nfunc (x *SetCurrentCatalog) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[25]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SetCurrentCatalog.ProtoReflect.Descriptor instead.\nfunc (*SetCurrentCatalog) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{25}\n}\n\nfunc (x *SetCurrentCatalog) GetCatalogName() string {\n\tif x != nil {\n\t\treturn x.CatalogName\n\t}\n\treturn \"\"\n}\n\n// See `spark.catalog.listCatalogs`\ntype ListCatalogs struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The pattern that the catalog name needs to match\n\tPattern *string `protobuf:\"bytes,1,opt,name=pattern,proto3,oneof\" json:\"pattern,omitempty\"`\n}\n\nfunc (x *ListCatalogs) Reset() {\n\t*x = ListCatalogs{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_catalog_proto_msgTypes[26]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ListCatalogs) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ListCatalogs) ProtoMessage() {}\n\nfunc (x *ListCatalogs) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_catalog_proto_msgTypes[26]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ListCatalogs.ProtoReflect.Descriptor instead.\nfunc (*ListCatalogs) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_catalog_proto_rawDescGZIP(), []int{26}\n}\n\nfunc (x *ListCatalogs) GetPattern() string {\n\tif x != nil && x.Pattern != nil {\n\t\treturn *x.Pattern\n\t}\n\treturn \"\"\n}\n\nvar File_spark_connect_catalog_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_catalog_proto_rawDesc = []byte{\n\t0x0a, 0x1b, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x1a, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d,\n\t0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2e, 0x70, 0x72,\n\t0x6f, 0x74, 0x6f, 0x22, 0xc6, 0x0e, 0x0a, 0x07, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x12,\n\t0x4b, 0x0a, 0x10, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x62,\n\t0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e,\n\t0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x63, 0x75, 0x72,\n\t0x72, 0x65, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x55, 0x0a, 0x14,\n\t0x73, 0x65, 0x74, 0x5f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61,\n\t0x62, 0x61, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x65, 0x74, 0x43, 0x75,\n\t0x72, 0x72, 0x65, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x48, 0x00, 0x52,\n\t0x12, 0x73, 0x65, 0x74, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62,\n\t0x61, 0x73, 0x65, 0x12, 0x45, 0x0a, 0x0e, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61,\n\t0x62, 0x61, 0x73, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x69, 0x73, 0x74,\n\t0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0d, 0x6c, 0x69, 0x73,\n\t0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x0b, 0x6c, 0x69,\n\t0x73, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x4c, 0x69, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x48, 0x00, 0x52, 0x0a, 0x6c, 0x69,\n\t0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x45, 0x0a, 0x0e, 0x6c, 0x69, 0x73, 0x74,\n\t0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x48, 0x00,\n\t0x52, 0x0d, 0x6c, 0x69, 0x73, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12,\n\t0x3f, 0x0a, 0x0c, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18,\n\t0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x73, 0x48, 0x00, 0x52, 0x0b, 0x6c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,\n\t0x12, 0x3f, 0x0a, 0x0c, 0x67, 0x65, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65,\n\t0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61,\n\t0x73, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x67, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73,\n\t0x65, 0x12, 0x36, 0x0a, 0x09, 0x67, 0x65, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x08,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52,\n\t0x08, 0x67, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x3f, 0x0a, 0x0c, 0x67, 0x65, 0x74,\n\t0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x47, 0x65, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x67,\n\t0x65, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0f, 0x64, 0x61,\n\t0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x0a, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x45, 0x78, 0x69, 0x73,\n\t0x74, 0x73, 0x48, 0x00, 0x52, 0x0e, 0x64, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x45, 0x78,\n\t0x69, 0x73, 0x74, 0x73, 0x12, 0x3f, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x78,\n\t0x69, 0x73, 0x74, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65,\n\t0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x48, 0x00, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x45,\n\t0x78, 0x69, 0x73, 0x74, 0x73, 0x12, 0x48, 0x0a, 0x0f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x65, 0x78, 0x69, 0x73, 0x74, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46,\n\t0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x48, 0x00, 0x52,\n\t0x0e, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x12,\n\t0x58, 0x0a, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e,\n\t0x61, 0x6c, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43,\n\t0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62,\n\t0x6c, 0x65, 0x48, 0x00, 0x52, 0x13, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65,\n\t0x72, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x3f, 0x0a, 0x0c, 0x63, 0x72, 0x65,\n\t0x61, 0x74, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x63,\n\t0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x43, 0x0a, 0x0e, 0x64, 0x72,\n\t0x6f, 0x70, 0x5f, 0x74, 0x65, 0x6d, 0x70, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x18, 0x0f, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x65, 0x6d, 0x70, 0x56, 0x69, 0x65, 0x77, 0x48,\n\t0x00, 0x52, 0x0c, 0x64, 0x72, 0x6f, 0x70, 0x54, 0x65, 0x6d, 0x70, 0x56, 0x69, 0x65, 0x77, 0x12,\n\t0x56, 0x0a, 0x15, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x74,\n\t0x65, 0x6d, 0x70, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x72, 0x6f, 0x70, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x54, 0x65, 0x6d, 0x70, 0x56, 0x69, 0x65,\n\t0x77, 0x48, 0x00, 0x52, 0x12, 0x64, 0x72, 0x6f, 0x70, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x54,\n\t0x65, 0x6d, 0x70, 0x56, 0x69, 0x65, 0x77, 0x12, 0x51, 0x0a, 0x12, 0x72, 0x65, 0x63, 0x6f, 0x76,\n\t0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x11, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72,\n\t0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x36, 0x0a, 0x09, 0x69, 0x73,\n\t0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x64, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x49, 0x73,\n\t0x43, 0x61, 0x63, 0x68, 0x65, 0x64, 0x48, 0x00, 0x52, 0x08, 0x69, 0x73, 0x43, 0x61, 0x63, 0x68,\n\t0x65, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x61, 0x62,\n\t0x6c, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x61, 0x63, 0x68, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65,\n\t0x12, 0x42, 0x0a, 0x0d, 0x75, 0x6e, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x6e, 0x63, 0x61, 0x63, 0x68, 0x65, 0x54,\n\t0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x75, 0x6e, 0x63, 0x61, 0x63, 0x68, 0x65, 0x54,\n\t0x61, 0x62, 0x6c, 0x65, 0x12, 0x3c, 0x0a, 0x0b, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x5f, 0x63, 0x61,\n\t0x63, 0x68, 0x65, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x43,\n\t0x61, 0x63, 0x68, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x43, 0x61, 0x63,\n\t0x68, 0x65, 0x12, 0x42, 0x0a, 0x0d, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x5f, 0x74, 0x61,\n\t0x62, 0x6c, 0x65, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73,\n\t0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73,\n\t0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x46, 0x0a, 0x0f, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73,\n\t0x68, 0x5f, 0x62, 0x79, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x48, 0x00, 0x52,\n\t0x0d, 0x72, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68, 0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x48,\n\t0x0a, 0x0f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f,\n\t0x67, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x43,\n\t0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e,\n\t0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x12, 0x52, 0x0a, 0x13, 0x73, 0x65, 0x74, 0x5f,\n\t0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18,\n\t0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x65, 0x74, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74,\n\t0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x48, 0x00, 0x52, 0x11, 0x73, 0x65, 0x74, 0x43, 0x75,\n\t0x72, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x12, 0x42, 0x0a, 0x0d,\n\t0x6c, 0x69, 0x73, 0x74, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73, 0x18, 0x1a, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73,\n\t0x48, 0x00, 0x52, 0x0c, 0x6c, 0x69, 0x73, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x73,\n\t0x42, 0x0a, 0x0a, 0x08, 0x63, 0x61, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x11, 0x0a, 0x0f,\n\t0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x22,\n\t0x2d, 0x0a, 0x12, 0x53, 0x65, 0x74, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x44, 0x61, 0x74,\n\t0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x3a,\n\t0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x12,\n\t0x1d, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0a,\n\t0x0a, 0x08, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x22, 0x61, 0x0a, 0x0a, 0x4c, 0x69,\n\t0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e,\n\t0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x64, 0x62, 0x4e,\n\t0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x1d, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72,\n\t0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65,\n\t0x72, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d,\n\t0x65, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x22, 0x64, 0x0a,\n\t0x0d, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1c,\n\t0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x1d, 0x0a, 0x07,\n\t0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52,\n\t0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f,\n\t0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x61, 0x74, 0x74,\n\t0x65, 0x72, 0x6e, 0x22, 0x56, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d,\n\t0x65, 0x12, 0x1c, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42,\n\t0x0a, 0x0a, 0x08, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x26, 0x0a, 0x0b, 0x47,\n\t0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62,\n\t0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e,\n\t0x61, 0x6d, 0x65, 0x22, 0x53, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12,\n\t0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c,\n\t0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08,\n\t0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x5c, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x46,\n\t0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c,\n\t0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x07,\n\t0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,\n\t0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x64,\n\t0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x29, 0x0a, 0x0e, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61,\n\t0x73, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e,\n\t0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d,\n\t0x65, 0x22, 0x56, 0x0a, 0x0b, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73,\n\t0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12,\n\t0x1c, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a,\n\t0x08, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x5f, 0x0a, 0x0e, 0x46, 0x75, 0x6e,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x69, 0x73, 0x74, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x66,\n\t0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65,\n\t0x12, 0x1c, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x00, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0a,\n\t0x0a, 0x08, 0x5f, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xc6, 0x02, 0x0a, 0x13, 0x43,\n\t0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62,\n\t0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d,\n\t0x65, 0x12, 0x17, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x88, 0x01, 0x01, 0x12, 0x34, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d,\n\t0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65,\n\t0x48, 0x02, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x49, 0x0a,\n\t0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43,\n\t0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x61, 0x62,\n\t0x6c, 0x65, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,\n\t0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,\n\t0x3a, 0x02, 0x38, 0x01, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x42, 0x09, 0x0a,\n\t0x07, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x73, 0x63, 0x68,\n\t0x65, 0x6d, 0x61, 0x22, 0xed, 0x02, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61,\n\t0x62, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61,\n\t0x6d, 0x65, 0x12, 0x17, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x73,\n\t0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x73,\n\t0x6f, 0x75, 0x72, 0x63, 0x65, 0x88, 0x01, 0x01, 0x12, 0x25, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63,\n\t0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52,\n\t0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12,\n\t0x34, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x03, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65,\n\t0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x41, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62,\n\t0x6c, 0x65, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,\n\t0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,\n\t0x3a, 0x02, 0x38, 0x01, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x42, 0x09, 0x0a,\n\t0x07, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x64, 0x65, 0x73,\n\t0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x73, 0x63, 0x68,\n\t0x65, 0x6d, 0x61, 0x22, 0x2b, 0x0a, 0x0c, 0x44, 0x72, 0x6f, 0x70, 0x54, 0x65, 0x6d, 0x70, 0x56,\n\t0x69, 0x65, 0x77, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x69, 0x65, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x69, 0x65, 0x77, 0x4e, 0x61, 0x6d, 0x65,\n\t0x22, 0x31, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x54, 0x65,\n\t0x6d, 0x70, 0x56, 0x69, 0x65, 0x77, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x69, 0x65, 0x77, 0x5f, 0x6e,\n\t0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x69, 0x65, 0x77, 0x4e,\n\t0x61, 0x6d, 0x65, 0x22, 0x32, 0x0a, 0x11, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x61,\n\t0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61,\n\t0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x29, 0x0a, 0x08, 0x49, 0x73, 0x43, 0x61, 0x63,\n\t0x68, 0x65, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61,\n\t0x6d, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x0a, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x61, 0x62, 0x6c,\n\t0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65,\n\t0x12, 0x45, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65,\n\t0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c,\n\t0x65, 0x76, 0x65, 0x6c, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c,\n\t0x65, 0x76, 0x65, 0x6c, 0x88, 0x01, 0x01, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x73, 0x74, 0x6f, 0x72,\n\t0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x22, 0x2d, 0x0a, 0x0c, 0x55, 0x6e, 0x63,\n\t0x61, 0x63, 0x68, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62,\n\t0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74,\n\t0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x0c, 0x0a, 0x0a, 0x43, 0x6c, 0x65, 0x61,\n\t0x72, 0x43, 0x61, 0x63, 0x68, 0x65, 0x22, 0x2d, 0x0a, 0x0c, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73,\n\t0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x23, 0x0a, 0x0d, 0x52, 0x65, 0x66, 0x72, 0x65, 0x73, 0x68,\n\t0x42, 0x79, 0x50, 0x61, 0x74, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x22, 0x10, 0x0a, 0x0e, 0x43, 0x75,\n\t0x72, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x22, 0x36, 0x0a, 0x11,\n\t0x53, 0x65, 0x74, 0x43, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f,\n\t0x67, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x5f, 0x6e, 0x61, 0x6d,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67,\n\t0x4e, 0x61, 0x6d, 0x65, 0x22, 0x39, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x61, 0x74, 0x61,\n\t0x6c, 0x6f, 0x67, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e,\n\t0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x42,\n\t0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74,\n\t0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65,\n\t0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_catalog_proto_rawDescOnce sync.Once\n\tfile_spark_connect_catalog_proto_rawDescData = file_spark_connect_catalog_proto_rawDesc\n)\n\nfunc file_spark_connect_catalog_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_catalog_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_catalog_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_catalog_proto_rawDescData)\n\t})\n\treturn file_spark_connect_catalog_proto_rawDescData\n}\n\nvar file_spark_connect_catalog_proto_msgTypes = make([]protoimpl.MessageInfo, 29)\nvar file_spark_connect_catalog_proto_goTypes = []interface{}{\n\t(*Catalog)(nil),             // 0: spark.connect.Catalog\n\t(*CurrentDatabase)(nil),     // 1: spark.connect.CurrentDatabase\n\t(*SetCurrentDatabase)(nil),  // 2: spark.connect.SetCurrentDatabase\n\t(*ListDatabases)(nil),       // 3: spark.connect.ListDatabases\n\t(*ListTables)(nil),          // 4: spark.connect.ListTables\n\t(*ListFunctions)(nil),       // 5: spark.connect.ListFunctions\n\t(*ListColumns)(nil),         // 6: spark.connect.ListColumns\n\t(*GetDatabase)(nil),         // 7: spark.connect.GetDatabase\n\t(*GetTable)(nil),            // 8: spark.connect.GetTable\n\t(*GetFunction)(nil),         // 9: spark.connect.GetFunction\n\t(*DatabaseExists)(nil),      // 10: spark.connect.DatabaseExists\n\t(*TableExists)(nil),         // 11: spark.connect.TableExists\n\t(*FunctionExists)(nil),      // 12: spark.connect.FunctionExists\n\t(*CreateExternalTable)(nil), // 13: spark.connect.CreateExternalTable\n\t(*CreateTable)(nil),         // 14: spark.connect.CreateTable\n\t(*DropTempView)(nil),        // 15: spark.connect.DropTempView\n\t(*DropGlobalTempView)(nil),  // 16: spark.connect.DropGlobalTempView\n\t(*RecoverPartitions)(nil),   // 17: spark.connect.RecoverPartitions\n\t(*IsCached)(nil),            // 18: spark.connect.IsCached\n\t(*CacheTable)(nil),          // 19: spark.connect.CacheTable\n\t(*UncacheTable)(nil),        // 20: spark.connect.UncacheTable\n\t(*ClearCache)(nil),          // 21: spark.connect.ClearCache\n\t(*RefreshTable)(nil),        // 22: spark.connect.RefreshTable\n\t(*RefreshByPath)(nil),       // 23: spark.connect.RefreshByPath\n\t(*CurrentCatalog)(nil),      // 24: spark.connect.CurrentCatalog\n\t(*SetCurrentCatalog)(nil),   // 25: spark.connect.SetCurrentCatalog\n\t(*ListCatalogs)(nil),        // 26: spark.connect.ListCatalogs\n\tnil,                         // 27: spark.connect.CreateExternalTable.OptionsEntry\n\tnil,                         // 28: spark.connect.CreateTable.OptionsEntry\n\t(*DataType)(nil),            // 29: spark.connect.DataType\n\t(*StorageLevel)(nil),        // 30: spark.connect.StorageLevel\n}\nvar file_spark_connect_catalog_proto_depIdxs = []int32{\n\t1,  // 0: spark.connect.Catalog.current_database:type_name -> spark.connect.CurrentDatabase\n\t2,  // 1: spark.connect.Catalog.set_current_database:type_name -> spark.connect.SetCurrentDatabase\n\t3,  // 2: spark.connect.Catalog.list_databases:type_name -> spark.connect.ListDatabases\n\t4,  // 3: spark.connect.Catalog.list_tables:type_name -> spark.connect.ListTables\n\t5,  // 4: spark.connect.Catalog.list_functions:type_name -> spark.connect.ListFunctions\n\t6,  // 5: spark.connect.Catalog.list_columns:type_name -> spark.connect.ListColumns\n\t7,  // 6: spark.connect.Catalog.get_database:type_name -> spark.connect.GetDatabase\n\t8,  // 7: spark.connect.Catalog.get_table:type_name -> spark.connect.GetTable\n\t9,  // 8: spark.connect.Catalog.get_function:type_name -> spark.connect.GetFunction\n\t10, // 9: spark.connect.Catalog.database_exists:type_name -> spark.connect.DatabaseExists\n\t11, // 10: spark.connect.Catalog.table_exists:type_name -> spark.connect.TableExists\n\t12, // 11: spark.connect.Catalog.function_exists:type_name -> spark.connect.FunctionExists\n\t13, // 12: spark.connect.Catalog.create_external_table:type_name -> spark.connect.CreateExternalTable\n\t14, // 13: spark.connect.Catalog.create_table:type_name -> spark.connect.CreateTable\n\t15, // 14: spark.connect.Catalog.drop_temp_view:type_name -> spark.connect.DropTempView\n\t16, // 15: spark.connect.Catalog.drop_global_temp_view:type_name -> spark.connect.DropGlobalTempView\n\t17, // 16: spark.connect.Catalog.recover_partitions:type_name -> spark.connect.RecoverPartitions\n\t18, // 17: spark.connect.Catalog.is_cached:type_name -> spark.connect.IsCached\n\t19, // 18: spark.connect.Catalog.cache_table:type_name -> spark.connect.CacheTable\n\t20, // 19: spark.connect.Catalog.uncache_table:type_name -> spark.connect.UncacheTable\n\t21, // 20: spark.connect.Catalog.clear_cache:type_name -> spark.connect.ClearCache\n\t22, // 21: spark.connect.Catalog.refresh_table:type_name -> spark.connect.RefreshTable\n\t23, // 22: spark.connect.Catalog.refresh_by_path:type_name -> spark.connect.RefreshByPath\n\t24, // 23: spark.connect.Catalog.current_catalog:type_name -> spark.connect.CurrentCatalog\n\t25, // 24: spark.connect.Catalog.set_current_catalog:type_name -> spark.connect.SetCurrentCatalog\n\t26, // 25: spark.connect.Catalog.list_catalogs:type_name -> spark.connect.ListCatalogs\n\t29, // 26: spark.connect.CreateExternalTable.schema:type_name -> spark.connect.DataType\n\t27, // 27: spark.connect.CreateExternalTable.options:type_name -> spark.connect.CreateExternalTable.OptionsEntry\n\t29, // 28: spark.connect.CreateTable.schema:type_name -> spark.connect.DataType\n\t28, // 29: spark.connect.CreateTable.options:type_name -> spark.connect.CreateTable.OptionsEntry\n\t30, // 30: spark.connect.CacheTable.storage_level:type_name -> spark.connect.StorageLevel\n\t31, // [31:31] is the sub-list for method output_type\n\t31, // [31:31] is the sub-list for method input_type\n\t31, // [31:31] is the sub-list for extension type_name\n\t31, // [31:31] is the sub-list for extension extendee\n\t0,  // [0:31] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_catalog_proto_init() }\nfunc file_spark_connect_catalog_proto_init() {\n\tif File_spark_connect_catalog_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_common_proto_init()\n\tfile_spark_connect_types_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_catalog_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Catalog); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CurrentDatabase); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SetCurrentDatabase); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ListDatabases); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ListTables); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ListFunctions); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ListColumns); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GetDatabase); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GetTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GetFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DatabaseExists); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*TableExists); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*FunctionExists); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CreateExternalTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CreateTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DropTempView); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DropGlobalTempView); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RecoverPartitions); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*IsCached); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CacheTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*UncacheTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ClearCache); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RefreshTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RefreshByPath); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CurrentCatalog); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SetCurrentCatalog); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_catalog_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ListCatalogs); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_catalog_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*Catalog_CurrentDatabase)(nil),\n\t\t(*Catalog_SetCurrentDatabase)(nil),\n\t\t(*Catalog_ListDatabases)(nil),\n\t\t(*Catalog_ListTables)(nil),\n\t\t(*Catalog_ListFunctions)(nil),\n\t\t(*Catalog_ListColumns)(nil),\n\t\t(*Catalog_GetDatabase)(nil),\n\t\t(*Catalog_GetTable)(nil),\n\t\t(*Catalog_GetFunction)(nil),\n\t\t(*Catalog_DatabaseExists)(nil),\n\t\t(*Catalog_TableExists)(nil),\n\t\t(*Catalog_FunctionExists)(nil),\n\t\t(*Catalog_CreateExternalTable)(nil),\n\t\t(*Catalog_CreateTable)(nil),\n\t\t(*Catalog_DropTempView)(nil),\n\t\t(*Catalog_DropGlobalTempView)(nil),\n\t\t(*Catalog_RecoverPartitions)(nil),\n\t\t(*Catalog_IsCached)(nil),\n\t\t(*Catalog_CacheTable)(nil),\n\t\t(*Catalog_UncacheTable)(nil),\n\t\t(*Catalog_ClearCache)(nil),\n\t\t(*Catalog_RefreshTable)(nil),\n\t\t(*Catalog_RefreshByPath)(nil),\n\t\t(*Catalog_CurrentCatalog)(nil),\n\t\t(*Catalog_SetCurrentCatalog)(nil),\n\t\t(*Catalog_ListCatalogs)(nil),\n\t}\n\tfile_spark_connect_catalog_proto_msgTypes[3].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[4].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[5].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[6].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[8].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[9].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[11].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[12].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[13].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[14].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[19].OneofWrappers = []interface{}{}\n\tfile_spark_connect_catalog_proto_msgTypes[26].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_catalog_proto_rawDesc,\n\t\t\tNumEnums:      0,\n\t\t\tNumMessages:   29,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_catalog_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_catalog_proto_depIdxs,\n\t\tMessageInfos:      file_spark_connect_catalog_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_catalog_proto = out.File\n\tfile_spark_connect_catalog_proto_rawDesc = nil\n\tfile_spark_connect_catalog_proto_goTypes = nil\n\tfile_spark_connect_catalog_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/commands.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/commands.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\tanypb \"google.golang.org/protobuf/types/known/anypb\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// The enum used for client side streaming query listener event\n// There is no QueryStartedEvent defined here,\n// it is added as a field in WriteStreamOperationStartResult\ntype StreamingQueryEventType int32\n\nconst (\n\tStreamingQueryEventType_QUERY_PROGRESS_UNSPECIFIED StreamingQueryEventType = 0\n\tStreamingQueryEventType_QUERY_PROGRESS_EVENT       StreamingQueryEventType = 1\n\tStreamingQueryEventType_QUERY_TERMINATED_EVENT     StreamingQueryEventType = 2\n\tStreamingQueryEventType_QUERY_IDLE_EVENT           StreamingQueryEventType = 3\n)\n\n// Enum value maps for StreamingQueryEventType.\nvar (\n\tStreamingQueryEventType_name = map[int32]string{\n\t\t0: \"QUERY_PROGRESS_UNSPECIFIED\",\n\t\t1: \"QUERY_PROGRESS_EVENT\",\n\t\t2: \"QUERY_TERMINATED_EVENT\",\n\t\t3: \"QUERY_IDLE_EVENT\",\n\t}\n\tStreamingQueryEventType_value = map[string]int32{\n\t\t\"QUERY_PROGRESS_UNSPECIFIED\": 0,\n\t\t\"QUERY_PROGRESS_EVENT\":       1,\n\t\t\"QUERY_TERMINATED_EVENT\":     2,\n\t\t\"QUERY_IDLE_EVENT\":           3,\n\t}\n)\n\nfunc (x StreamingQueryEventType) Enum() *StreamingQueryEventType {\n\tp := new(StreamingQueryEventType)\n\t*p = x\n\treturn p\n}\n\nfunc (x StreamingQueryEventType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (StreamingQueryEventType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_commands_proto_enumTypes[0].Descriptor()\n}\n\nfunc (StreamingQueryEventType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_commands_proto_enumTypes[0]\n}\n\nfunc (x StreamingQueryEventType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use StreamingQueryEventType.Descriptor instead.\nfunc (StreamingQueryEventType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{0}\n}\n\ntype WriteOperation_SaveMode int32\n\nconst (\n\tWriteOperation_SAVE_MODE_UNSPECIFIED     WriteOperation_SaveMode = 0\n\tWriteOperation_SAVE_MODE_APPEND          WriteOperation_SaveMode = 1\n\tWriteOperation_SAVE_MODE_OVERWRITE       WriteOperation_SaveMode = 2\n\tWriteOperation_SAVE_MODE_ERROR_IF_EXISTS WriteOperation_SaveMode = 3\n\tWriteOperation_SAVE_MODE_IGNORE          WriteOperation_SaveMode = 4\n)\n\n// Enum value maps for WriteOperation_SaveMode.\nvar (\n\tWriteOperation_SaveMode_name = map[int32]string{\n\t\t0: \"SAVE_MODE_UNSPECIFIED\",\n\t\t1: \"SAVE_MODE_APPEND\",\n\t\t2: \"SAVE_MODE_OVERWRITE\",\n\t\t3: \"SAVE_MODE_ERROR_IF_EXISTS\",\n\t\t4: \"SAVE_MODE_IGNORE\",\n\t}\n\tWriteOperation_SaveMode_value = map[string]int32{\n\t\t\"SAVE_MODE_UNSPECIFIED\":     0,\n\t\t\"SAVE_MODE_APPEND\":          1,\n\t\t\"SAVE_MODE_OVERWRITE\":       2,\n\t\t\"SAVE_MODE_ERROR_IF_EXISTS\": 3,\n\t\t\"SAVE_MODE_IGNORE\":          4,\n\t}\n)\n\nfunc (x WriteOperation_SaveMode) Enum() *WriteOperation_SaveMode {\n\tp := new(WriteOperation_SaveMode)\n\t*p = x\n\treturn p\n}\n\nfunc (x WriteOperation_SaveMode) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (WriteOperation_SaveMode) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_commands_proto_enumTypes[1].Descriptor()\n}\n\nfunc (WriteOperation_SaveMode) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_commands_proto_enumTypes[1]\n}\n\nfunc (x WriteOperation_SaveMode) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use WriteOperation_SaveMode.Descriptor instead.\nfunc (WriteOperation_SaveMode) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{3, 0}\n}\n\ntype WriteOperation_SaveTable_TableSaveMethod int32\n\nconst (\n\tWriteOperation_SaveTable_TABLE_SAVE_METHOD_UNSPECIFIED   WriteOperation_SaveTable_TableSaveMethod = 0\n\tWriteOperation_SaveTable_TABLE_SAVE_METHOD_SAVE_AS_TABLE WriteOperation_SaveTable_TableSaveMethod = 1\n\tWriteOperation_SaveTable_TABLE_SAVE_METHOD_INSERT_INTO   WriteOperation_SaveTable_TableSaveMethod = 2\n)\n\n// Enum value maps for WriteOperation_SaveTable_TableSaveMethod.\nvar (\n\tWriteOperation_SaveTable_TableSaveMethod_name = map[int32]string{\n\t\t0: \"TABLE_SAVE_METHOD_UNSPECIFIED\",\n\t\t1: \"TABLE_SAVE_METHOD_SAVE_AS_TABLE\",\n\t\t2: \"TABLE_SAVE_METHOD_INSERT_INTO\",\n\t}\n\tWriteOperation_SaveTable_TableSaveMethod_value = map[string]int32{\n\t\t\"TABLE_SAVE_METHOD_UNSPECIFIED\":   0,\n\t\t\"TABLE_SAVE_METHOD_SAVE_AS_TABLE\": 1,\n\t\t\"TABLE_SAVE_METHOD_INSERT_INTO\":   2,\n\t}\n)\n\nfunc (x WriteOperation_SaveTable_TableSaveMethod) Enum() *WriteOperation_SaveTable_TableSaveMethod {\n\tp := new(WriteOperation_SaveTable_TableSaveMethod)\n\t*p = x\n\treturn p\n}\n\nfunc (x WriteOperation_SaveTable_TableSaveMethod) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (WriteOperation_SaveTable_TableSaveMethod) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_commands_proto_enumTypes[2].Descriptor()\n}\n\nfunc (WriteOperation_SaveTable_TableSaveMethod) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_commands_proto_enumTypes[2]\n}\n\nfunc (x WriteOperation_SaveTable_TableSaveMethod) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use WriteOperation_SaveTable_TableSaveMethod.Descriptor instead.\nfunc (WriteOperation_SaveTable_TableSaveMethod) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{3, 1, 0}\n}\n\ntype WriteOperationV2_Mode int32\n\nconst (\n\tWriteOperationV2_MODE_UNSPECIFIED          WriteOperationV2_Mode = 0\n\tWriteOperationV2_MODE_CREATE               WriteOperationV2_Mode = 1\n\tWriteOperationV2_MODE_OVERWRITE            WriteOperationV2_Mode = 2\n\tWriteOperationV2_MODE_OVERWRITE_PARTITIONS WriteOperationV2_Mode = 3\n\tWriteOperationV2_MODE_APPEND               WriteOperationV2_Mode = 4\n\tWriteOperationV2_MODE_REPLACE              WriteOperationV2_Mode = 5\n\tWriteOperationV2_MODE_CREATE_OR_REPLACE    WriteOperationV2_Mode = 6\n)\n\n// Enum value maps for WriteOperationV2_Mode.\nvar (\n\tWriteOperationV2_Mode_name = map[int32]string{\n\t\t0: \"MODE_UNSPECIFIED\",\n\t\t1: \"MODE_CREATE\",\n\t\t2: \"MODE_OVERWRITE\",\n\t\t3: \"MODE_OVERWRITE_PARTITIONS\",\n\t\t4: \"MODE_APPEND\",\n\t\t5: \"MODE_REPLACE\",\n\t\t6: \"MODE_CREATE_OR_REPLACE\",\n\t}\n\tWriteOperationV2_Mode_value = map[string]int32{\n\t\t\"MODE_UNSPECIFIED\":          0,\n\t\t\"MODE_CREATE\":               1,\n\t\t\"MODE_OVERWRITE\":            2,\n\t\t\"MODE_OVERWRITE_PARTITIONS\": 3,\n\t\t\"MODE_APPEND\":               4,\n\t\t\"MODE_REPLACE\":              5,\n\t\t\"MODE_CREATE_OR_REPLACE\":    6,\n\t}\n)\n\nfunc (x WriteOperationV2_Mode) Enum() *WriteOperationV2_Mode {\n\tp := new(WriteOperationV2_Mode)\n\t*p = x\n\treturn p\n}\n\nfunc (x WriteOperationV2_Mode) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (WriteOperationV2_Mode) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_commands_proto_enumTypes[3].Descriptor()\n}\n\nfunc (WriteOperationV2_Mode) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_commands_proto_enumTypes[3]\n}\n\nfunc (x WriteOperationV2_Mode) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use WriteOperationV2_Mode.Descriptor instead.\nfunc (WriteOperationV2_Mode) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{4, 0}\n}\n\n// A [[Command]] is an operation that is executed by the server that does not directly consume or\n// produce a relational result.\ntype Command struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to CommandType:\n\t//\n\t//\t*Command_RegisterFunction\n\t//\t*Command_WriteOperation\n\t//\t*Command_CreateDataframeView\n\t//\t*Command_WriteOperationV2\n\t//\t*Command_SqlCommand\n\t//\t*Command_WriteStreamOperationStart\n\t//\t*Command_StreamingQueryCommand\n\t//\t*Command_GetResourcesCommand\n\t//\t*Command_StreamingQueryManagerCommand\n\t//\t*Command_RegisterTableFunction\n\t//\t*Command_StreamingQueryListenerBusCommand\n\t//\t*Command_RegisterDataSource\n\t//\t*Command_CreateResourceProfileCommand\n\t//\t*Command_CheckpointCommand\n\t//\t*Command_RemoveCachedRemoteRelationCommand\n\t//\t*Command_MergeIntoTableCommand\n\t//\t*Command_MlCommand\n\t//\t*Command_ExecuteExternalCommand\n\t//\t*Command_PipelineCommand\n\t//\t*Command_Extension\n\tCommandType isCommand_CommandType `protobuf_oneof:\"command_type\"`\n}\n\nfunc (x *Command) Reset() {\n\t*x = Command{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Command) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Command) ProtoMessage() {}\n\nfunc (x *Command) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Command.ProtoReflect.Descriptor instead.\nfunc (*Command) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *Command) GetCommandType() isCommand_CommandType {\n\tif m != nil {\n\t\treturn m.CommandType\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetRegisterFunction() *CommonInlineUserDefinedFunction {\n\tif x, ok := x.GetCommandType().(*Command_RegisterFunction); ok {\n\t\treturn x.RegisterFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetWriteOperation() *WriteOperation {\n\tif x, ok := x.GetCommandType().(*Command_WriteOperation); ok {\n\t\treturn x.WriteOperation\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetCreateDataframeView() *CreateDataFrameViewCommand {\n\tif x, ok := x.GetCommandType().(*Command_CreateDataframeView); ok {\n\t\treturn x.CreateDataframeView\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetWriteOperationV2() *WriteOperationV2 {\n\tif x, ok := x.GetCommandType().(*Command_WriteOperationV2); ok {\n\t\treturn x.WriteOperationV2\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetSqlCommand() *SqlCommand {\n\tif x, ok := x.GetCommandType().(*Command_SqlCommand); ok {\n\t\treturn x.SqlCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetWriteStreamOperationStart() *WriteStreamOperationStart {\n\tif x, ok := x.GetCommandType().(*Command_WriteStreamOperationStart); ok {\n\t\treturn x.WriteStreamOperationStart\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetStreamingQueryCommand() *StreamingQueryCommand {\n\tif x, ok := x.GetCommandType().(*Command_StreamingQueryCommand); ok {\n\t\treturn x.StreamingQueryCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetGetResourcesCommand() *GetResourcesCommand {\n\tif x, ok := x.GetCommandType().(*Command_GetResourcesCommand); ok {\n\t\treturn x.GetResourcesCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetStreamingQueryManagerCommand() *StreamingQueryManagerCommand {\n\tif x, ok := x.GetCommandType().(*Command_StreamingQueryManagerCommand); ok {\n\t\treturn x.StreamingQueryManagerCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetRegisterTableFunction() *CommonInlineUserDefinedTableFunction {\n\tif x, ok := x.GetCommandType().(*Command_RegisterTableFunction); ok {\n\t\treturn x.RegisterTableFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetStreamingQueryListenerBusCommand() *StreamingQueryListenerBusCommand {\n\tif x, ok := x.GetCommandType().(*Command_StreamingQueryListenerBusCommand); ok {\n\t\treturn x.StreamingQueryListenerBusCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetRegisterDataSource() *CommonInlineUserDefinedDataSource {\n\tif x, ok := x.GetCommandType().(*Command_RegisterDataSource); ok {\n\t\treturn x.RegisterDataSource\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetCreateResourceProfileCommand() *CreateResourceProfileCommand {\n\tif x, ok := x.GetCommandType().(*Command_CreateResourceProfileCommand); ok {\n\t\treturn x.CreateResourceProfileCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetCheckpointCommand() *CheckpointCommand {\n\tif x, ok := x.GetCommandType().(*Command_CheckpointCommand); ok {\n\t\treturn x.CheckpointCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetRemoveCachedRemoteRelationCommand() *RemoveCachedRemoteRelationCommand {\n\tif x, ok := x.GetCommandType().(*Command_RemoveCachedRemoteRelationCommand); ok {\n\t\treturn x.RemoveCachedRemoteRelationCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetMergeIntoTableCommand() *MergeIntoTableCommand {\n\tif x, ok := x.GetCommandType().(*Command_MergeIntoTableCommand); ok {\n\t\treturn x.MergeIntoTableCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetMlCommand() *MlCommand {\n\tif x, ok := x.GetCommandType().(*Command_MlCommand); ok {\n\t\treturn x.MlCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetExecuteExternalCommand() *ExecuteExternalCommand {\n\tif x, ok := x.GetCommandType().(*Command_ExecuteExternalCommand); ok {\n\t\treturn x.ExecuteExternalCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetPipelineCommand() *PipelineCommand {\n\tif x, ok := x.GetCommandType().(*Command_PipelineCommand); ok {\n\t\treturn x.PipelineCommand\n\t}\n\treturn nil\n}\n\nfunc (x *Command) GetExtension() *anypb.Any {\n\tif x, ok := x.GetCommandType().(*Command_Extension); ok {\n\t\treturn x.Extension\n\t}\n\treturn nil\n}\n\ntype isCommand_CommandType interface {\n\tisCommand_CommandType()\n}\n\ntype Command_RegisterFunction struct {\n\tRegisterFunction *CommonInlineUserDefinedFunction `protobuf:\"bytes,1,opt,name=register_function,json=registerFunction,proto3,oneof\"`\n}\n\ntype Command_WriteOperation struct {\n\tWriteOperation *WriteOperation `protobuf:\"bytes,2,opt,name=write_operation,json=writeOperation,proto3,oneof\"`\n}\n\ntype Command_CreateDataframeView struct {\n\tCreateDataframeView *CreateDataFrameViewCommand `protobuf:\"bytes,3,opt,name=create_dataframe_view,json=createDataframeView,proto3,oneof\"`\n}\n\ntype Command_WriteOperationV2 struct {\n\tWriteOperationV2 *WriteOperationV2 `protobuf:\"bytes,4,opt,name=write_operation_v2,json=writeOperationV2,proto3,oneof\"`\n}\n\ntype Command_SqlCommand struct {\n\tSqlCommand *SqlCommand `protobuf:\"bytes,5,opt,name=sql_command,json=sqlCommand,proto3,oneof\"`\n}\n\ntype Command_WriteStreamOperationStart struct {\n\tWriteStreamOperationStart *WriteStreamOperationStart `protobuf:\"bytes,6,opt,name=write_stream_operation_start,json=writeStreamOperationStart,proto3,oneof\"`\n}\n\ntype Command_StreamingQueryCommand struct {\n\tStreamingQueryCommand *StreamingQueryCommand `protobuf:\"bytes,7,opt,name=streaming_query_command,json=streamingQueryCommand,proto3,oneof\"`\n}\n\ntype Command_GetResourcesCommand struct {\n\tGetResourcesCommand *GetResourcesCommand `protobuf:\"bytes,8,opt,name=get_resources_command,json=getResourcesCommand,proto3,oneof\"`\n}\n\ntype Command_StreamingQueryManagerCommand struct {\n\tStreamingQueryManagerCommand *StreamingQueryManagerCommand `protobuf:\"bytes,9,opt,name=streaming_query_manager_command,json=streamingQueryManagerCommand,proto3,oneof\"`\n}\n\ntype Command_RegisterTableFunction struct {\n\tRegisterTableFunction *CommonInlineUserDefinedTableFunction `protobuf:\"bytes,10,opt,name=register_table_function,json=registerTableFunction,proto3,oneof\"`\n}\n\ntype Command_StreamingQueryListenerBusCommand struct {\n\tStreamingQueryListenerBusCommand *StreamingQueryListenerBusCommand `protobuf:\"bytes,11,opt,name=streaming_query_listener_bus_command,json=streamingQueryListenerBusCommand,proto3,oneof\"`\n}\n\ntype Command_RegisterDataSource struct {\n\tRegisterDataSource *CommonInlineUserDefinedDataSource `protobuf:\"bytes,12,opt,name=register_data_source,json=registerDataSource,proto3,oneof\"`\n}\n\ntype Command_CreateResourceProfileCommand struct {\n\tCreateResourceProfileCommand *CreateResourceProfileCommand `protobuf:\"bytes,13,opt,name=create_resource_profile_command,json=createResourceProfileCommand,proto3,oneof\"`\n}\n\ntype Command_CheckpointCommand struct {\n\tCheckpointCommand *CheckpointCommand `protobuf:\"bytes,14,opt,name=checkpoint_command,json=checkpointCommand,proto3,oneof\"`\n}\n\ntype Command_RemoveCachedRemoteRelationCommand struct {\n\tRemoveCachedRemoteRelationCommand *RemoveCachedRemoteRelationCommand `protobuf:\"bytes,15,opt,name=remove_cached_remote_relation_command,json=removeCachedRemoteRelationCommand,proto3,oneof\"`\n}\n\ntype Command_MergeIntoTableCommand struct {\n\tMergeIntoTableCommand *MergeIntoTableCommand `protobuf:\"bytes,16,opt,name=merge_into_table_command,json=mergeIntoTableCommand,proto3,oneof\"`\n}\n\ntype Command_MlCommand struct {\n\tMlCommand *MlCommand `protobuf:\"bytes,17,opt,name=ml_command,json=mlCommand,proto3,oneof\"`\n}\n\ntype Command_ExecuteExternalCommand struct {\n\tExecuteExternalCommand *ExecuteExternalCommand `protobuf:\"bytes,18,opt,name=execute_external_command,json=executeExternalCommand,proto3,oneof\"`\n}\n\ntype Command_PipelineCommand struct {\n\tPipelineCommand *PipelineCommand `protobuf:\"bytes,19,opt,name=pipeline_command,json=pipelineCommand,proto3,oneof\"`\n}\n\ntype Command_Extension struct {\n\t// This field is used to mark extensions to the protocol. When plugins generate arbitrary\n\t// Commands they can add them here. During the planning the correct resolution is done.\n\tExtension *anypb.Any `protobuf:\"bytes,999,opt,name=extension,proto3,oneof\"`\n}\n\nfunc (*Command_RegisterFunction) isCommand_CommandType() {}\n\nfunc (*Command_WriteOperation) isCommand_CommandType() {}\n\nfunc (*Command_CreateDataframeView) isCommand_CommandType() {}\n\nfunc (*Command_WriteOperationV2) isCommand_CommandType() {}\n\nfunc (*Command_SqlCommand) isCommand_CommandType() {}\n\nfunc (*Command_WriteStreamOperationStart) isCommand_CommandType() {}\n\nfunc (*Command_StreamingQueryCommand) isCommand_CommandType() {}\n\nfunc (*Command_GetResourcesCommand) isCommand_CommandType() {}\n\nfunc (*Command_StreamingQueryManagerCommand) isCommand_CommandType() {}\n\nfunc (*Command_RegisterTableFunction) isCommand_CommandType() {}\n\nfunc (*Command_StreamingQueryListenerBusCommand) isCommand_CommandType() {}\n\nfunc (*Command_RegisterDataSource) isCommand_CommandType() {}\n\nfunc (*Command_CreateResourceProfileCommand) isCommand_CommandType() {}\n\nfunc (*Command_CheckpointCommand) isCommand_CommandType() {}\n\nfunc (*Command_RemoveCachedRemoteRelationCommand) isCommand_CommandType() {}\n\nfunc (*Command_MergeIntoTableCommand) isCommand_CommandType() {}\n\nfunc (*Command_MlCommand) isCommand_CommandType() {}\n\nfunc (*Command_ExecuteExternalCommand) isCommand_CommandType() {}\n\nfunc (*Command_PipelineCommand) isCommand_CommandType() {}\n\nfunc (*Command_Extension) isCommand_CommandType() {}\n\n// A SQL Command is used to trigger the eager evaluation of SQL commands in Spark.\n//\n// When the SQL provide as part of the message is a command it will be immediately evaluated\n// and the result will be collected and returned as part of a LocalRelation. If the result is\n// not a command, the operation will simply return a SQL Relation. This allows the client to be\n// almost oblivious to the server-side behavior.\ntype SqlCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) SQL Query.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/commands.proto.\n\tSql string `protobuf:\"bytes,1,opt,name=sql,proto3\" json:\"sql,omitempty\"`\n\t// (Optional) A map of parameter names to literal expressions.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/commands.proto.\n\tArgs map[string]*Expression_Literal `protobuf:\"bytes,2,rep,name=args,proto3\" json:\"args,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A sequence of literal expressions for positional parameters in the SQL query text.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/commands.proto.\n\tPosArgs []*Expression_Literal `protobuf:\"bytes,3,rep,name=pos_args,json=posArgs,proto3\" json:\"pos_args,omitempty\"`\n\t// (Optional) A map of parameter names to expressions.\n\t// It cannot coexist with `pos_arguments`.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/commands.proto.\n\tNamedArguments map[string]*Expression `protobuf:\"bytes,4,rep,name=named_arguments,json=namedArguments,proto3\" json:\"named_arguments,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A sequence of expressions for positional parameters in the SQL query text.\n\t// It cannot coexist with `named_arguments`.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/commands.proto.\n\tPosArguments []*Expression `protobuf:\"bytes,5,rep,name=pos_arguments,json=posArguments,proto3\" json:\"pos_arguments,omitempty\"`\n\t// (Optional) The relation that this SQL command will be built on.\n\tInput *Relation `protobuf:\"bytes,6,opt,name=input,proto3\" json:\"input,omitempty\"`\n}\n\nfunc (x *SqlCommand) Reset() {\n\t*x = SqlCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SqlCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SqlCommand) ProtoMessage() {}\n\nfunc (x *SqlCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SqlCommand.ProtoReflect.Descriptor instead.\nfunc (*SqlCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{1}\n}\n\n// Deprecated: Marked as deprecated in spark/connect/commands.proto.\nfunc (x *SqlCommand) GetSql() string {\n\tif x != nil {\n\t\treturn x.Sql\n\t}\n\treturn \"\"\n}\n\n// Deprecated: Marked as deprecated in spark/connect/commands.proto.\nfunc (x *SqlCommand) GetArgs() map[string]*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Args\n\t}\n\treturn nil\n}\n\n// Deprecated: Marked as deprecated in spark/connect/commands.proto.\nfunc (x *SqlCommand) GetPosArgs() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.PosArgs\n\t}\n\treturn nil\n}\n\n// Deprecated: Marked as deprecated in spark/connect/commands.proto.\nfunc (x *SqlCommand) GetNamedArguments() map[string]*Expression {\n\tif x != nil {\n\t\treturn x.NamedArguments\n\t}\n\treturn nil\n}\n\n// Deprecated: Marked as deprecated in spark/connect/commands.proto.\nfunc (x *SqlCommand) GetPosArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.PosArguments\n\t}\n\treturn nil\n}\n\nfunc (x *SqlCommand) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\n// A command that can create DataFrame global temp view or local temp view.\ntype CreateDataFrameViewCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The relation that this view will be built on.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) View name.\n\tName string `protobuf:\"bytes,2,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Required) Whether this is global temp view or local temp view.\n\tIsGlobal bool `protobuf:\"varint,3,opt,name=is_global,json=isGlobal,proto3\" json:\"is_global,omitempty\"`\n\t// (Required)\n\t//\n\t// If true, and if the view already exists, updates it; if false, and if the view\n\t// already exists, throws exception.\n\tReplace bool `protobuf:\"varint,4,opt,name=replace,proto3\" json:\"replace,omitempty\"`\n}\n\nfunc (x *CreateDataFrameViewCommand) Reset() {\n\t*x = CreateDataFrameViewCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CreateDataFrameViewCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CreateDataFrameViewCommand) ProtoMessage() {}\n\nfunc (x *CreateDataFrameViewCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CreateDataFrameViewCommand.ProtoReflect.Descriptor instead.\nfunc (*CreateDataFrameViewCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *CreateDataFrameViewCommand) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *CreateDataFrameViewCommand) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *CreateDataFrameViewCommand) GetIsGlobal() bool {\n\tif x != nil {\n\t\treturn x.IsGlobal\n\t}\n\treturn false\n}\n\nfunc (x *CreateDataFrameViewCommand) GetReplace() bool {\n\tif x != nil {\n\t\treturn x.Replace\n\t}\n\treturn false\n}\n\n// As writes are not directly handled during analysis and planning, they are modeled as commands.\ntype WriteOperation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The output of the `input` relation will be persisted according to the options.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Format value according to the Spark documentation. Examples are: text, parquet, delta.\n\tSource *string `protobuf:\"bytes,2,opt,name=source,proto3,oneof\" json:\"source,omitempty\"`\n\t// (Optional)\n\t//\n\t// The destination of the write operation can be either a path or a table.\n\t// If the destination is neither a path nor a table, such as jdbc and noop,\n\t// the `save_type` should not be set.\n\t//\n\t// Types that are assignable to SaveType:\n\t//\n\t//\t*WriteOperation_Path\n\t//\t*WriteOperation_Table\n\tSaveType isWriteOperation_SaveType `protobuf_oneof:\"save_type\"`\n\t// (Required) the save mode.\n\tMode WriteOperation_SaveMode `protobuf:\"varint,5,opt,name=mode,proto3,enum=spark.connect.WriteOperation_SaveMode\" json:\"mode,omitempty\"`\n\t// (Optional) List of columns to sort the output by.\n\tSortColumnNames []string `protobuf:\"bytes,6,rep,name=sort_column_names,json=sortColumnNames,proto3\" json:\"sort_column_names,omitempty\"`\n\t// (Optional) List of columns for partitioning.\n\tPartitioningColumns []string `protobuf:\"bytes,7,rep,name=partitioning_columns,json=partitioningColumns,proto3\" json:\"partitioning_columns,omitempty\"`\n\t// (Optional) Bucketing specification. Bucketing must set the number of buckets and the columns\n\t// to bucket by.\n\tBucketBy *WriteOperation_BucketBy `protobuf:\"bytes,8,opt,name=bucket_by,json=bucketBy,proto3\" json:\"bucket_by,omitempty\"`\n\t// (Optional) A list of configuration options.\n\tOptions map[string]string `protobuf:\"bytes,9,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) Columns used for clustering the table.\n\tClusteringColumns []string `protobuf:\"bytes,10,rep,name=clustering_columns,json=clusteringColumns,proto3\" json:\"clustering_columns,omitempty\"`\n}\n\nfunc (x *WriteOperation) Reset() {\n\t*x = WriteOperation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteOperation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteOperation) ProtoMessage() {}\n\nfunc (x *WriteOperation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteOperation.ProtoReflect.Descriptor instead.\nfunc (*WriteOperation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *WriteOperation) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetSource() string {\n\tif x != nil && x.Source != nil {\n\t\treturn *x.Source\n\t}\n\treturn \"\"\n}\n\nfunc (m *WriteOperation) GetSaveType() isWriteOperation_SaveType {\n\tif m != nil {\n\t\treturn m.SaveType\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetPath() string {\n\tif x, ok := x.GetSaveType().(*WriteOperation_Path); ok {\n\t\treturn x.Path\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteOperation) GetTable() *WriteOperation_SaveTable {\n\tif x, ok := x.GetSaveType().(*WriteOperation_Table); ok {\n\t\treturn x.Table\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetMode() WriteOperation_SaveMode {\n\tif x != nil {\n\t\treturn x.Mode\n\t}\n\treturn WriteOperation_SAVE_MODE_UNSPECIFIED\n}\n\nfunc (x *WriteOperation) GetSortColumnNames() []string {\n\tif x != nil {\n\t\treturn x.SortColumnNames\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetPartitioningColumns() []string {\n\tif x != nil {\n\t\treturn x.PartitioningColumns\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetBucketBy() *WriteOperation_BucketBy {\n\tif x != nil {\n\t\treturn x.BucketBy\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation) GetClusteringColumns() []string {\n\tif x != nil {\n\t\treturn x.ClusteringColumns\n\t}\n\treturn nil\n}\n\ntype isWriteOperation_SaveType interface {\n\tisWriteOperation_SaveType()\n}\n\ntype WriteOperation_Path struct {\n\tPath string `protobuf:\"bytes,3,opt,name=path,proto3,oneof\"`\n}\n\ntype WriteOperation_Table struct {\n\tTable *WriteOperation_SaveTable `protobuf:\"bytes,4,opt,name=table,proto3,oneof\"`\n}\n\nfunc (*WriteOperation_Path) isWriteOperation_SaveType() {}\n\nfunc (*WriteOperation_Table) isWriteOperation_SaveType() {}\n\n// As writes are not directly handled during analysis and planning, they are modeled as commands.\ntype WriteOperationV2 struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The output of the `input` relation will be persisted according to the options.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The destination of the write operation must be either a path or a table.\n\tTableName string `protobuf:\"bytes,2,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Optional) A provider for the underlying output data source. Spark's default catalog supports\n\t// \"parquet\", \"json\", etc.\n\tProvider *string `protobuf:\"bytes,3,opt,name=provider,proto3,oneof\" json:\"provider,omitempty\"`\n\t// (Optional) List of columns for partitioning for output table created by `create`,\n\t// `createOrReplace`, or `replace`\n\tPartitioningColumns []*Expression `protobuf:\"bytes,4,rep,name=partitioning_columns,json=partitioningColumns,proto3\" json:\"partitioning_columns,omitempty\"`\n\t// (Optional) A list of configuration options.\n\tOptions map[string]string `protobuf:\"bytes,5,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A list of table properties.\n\tTableProperties map[string]string `protobuf:\"bytes,6,rep,name=table_properties,json=tableProperties,proto3\" json:\"table_properties,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Required) Write mode.\n\tMode WriteOperationV2_Mode `protobuf:\"varint,7,opt,name=mode,proto3,enum=spark.connect.WriteOperationV2_Mode\" json:\"mode,omitempty\"`\n\t// (Optional) A condition for overwrite saving mode\n\tOverwriteCondition *Expression `protobuf:\"bytes,8,opt,name=overwrite_condition,json=overwriteCondition,proto3\" json:\"overwrite_condition,omitempty\"`\n\t// (Optional) Columns used for clustering the table.\n\tClusteringColumns []string `protobuf:\"bytes,9,rep,name=clustering_columns,json=clusteringColumns,proto3\" json:\"clustering_columns,omitempty\"`\n}\n\nfunc (x *WriteOperationV2) Reset() {\n\t*x = WriteOperationV2{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteOperationV2) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteOperationV2) ProtoMessage() {}\n\nfunc (x *WriteOperationV2) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteOperationV2.ProtoReflect.Descriptor instead.\nfunc (*WriteOperationV2) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *WriteOperationV2) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperationV2) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteOperationV2) GetProvider() string {\n\tif x != nil && x.Provider != nil {\n\t\treturn *x.Provider\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteOperationV2) GetPartitioningColumns() []*Expression {\n\tif x != nil {\n\t\treturn x.PartitioningColumns\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperationV2) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperationV2) GetTableProperties() map[string]string {\n\tif x != nil {\n\t\treturn x.TableProperties\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperationV2) GetMode() WriteOperationV2_Mode {\n\tif x != nil {\n\t\treturn x.Mode\n\t}\n\treturn WriteOperationV2_MODE_UNSPECIFIED\n}\n\nfunc (x *WriteOperationV2) GetOverwriteCondition() *Expression {\n\tif x != nil {\n\t\treturn x.OverwriteCondition\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperationV2) GetClusteringColumns() []string {\n\tif x != nil {\n\t\treturn x.ClusteringColumns\n\t}\n\treturn nil\n}\n\n// Starts write stream operation as streaming query. Query ID and Run ID of the streaming\n// query are returned.\ntype WriteStreamOperationStart struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The output of the `input` streaming relation will be written.\n\tInput                   *Relation         `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\tFormat                  string            `protobuf:\"bytes,2,opt,name=format,proto3\" json:\"format,omitempty\"`\n\tOptions                 map[string]string `protobuf:\"bytes,3,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\tPartitioningColumnNames []string          `protobuf:\"bytes,4,rep,name=partitioning_column_names,json=partitioningColumnNames,proto3\" json:\"partitioning_column_names,omitempty\"`\n\t// Types that are assignable to Trigger:\n\t//\n\t//\t*WriteStreamOperationStart_ProcessingTimeInterval\n\t//\t*WriteStreamOperationStart_AvailableNow\n\t//\t*WriteStreamOperationStart_Once\n\t//\t*WriteStreamOperationStart_ContinuousCheckpointInterval\n\tTrigger    isWriteStreamOperationStart_Trigger `protobuf_oneof:\"trigger\"`\n\tOutputMode string                              `protobuf:\"bytes,9,opt,name=output_mode,json=outputMode,proto3\" json:\"output_mode,omitempty\"`\n\tQueryName  string                              `protobuf:\"bytes,10,opt,name=query_name,json=queryName,proto3\" json:\"query_name,omitempty\"`\n\t// The destination is optional. When set, it can be a path or a table name.\n\t//\n\t// Types that are assignable to SinkDestination:\n\t//\n\t//\t*WriteStreamOperationStart_Path\n\t//\t*WriteStreamOperationStart_TableName\n\tSinkDestination isWriteStreamOperationStart_SinkDestination `protobuf_oneof:\"sink_destination\"`\n\tForeachWriter   *StreamingForeachFunction                   `protobuf:\"bytes,13,opt,name=foreach_writer,json=foreachWriter,proto3\" json:\"foreach_writer,omitempty\"`\n\tForeachBatch    *StreamingForeachFunction                   `protobuf:\"bytes,14,opt,name=foreach_batch,json=foreachBatch,proto3\" json:\"foreach_batch,omitempty\"`\n\t// (Optional) Columns used for clustering the table.\n\tClusteringColumnNames []string `protobuf:\"bytes,15,rep,name=clustering_column_names,json=clusteringColumnNames,proto3\" json:\"clustering_column_names,omitempty\"`\n}\n\nfunc (x *WriteStreamOperationStart) Reset() {\n\t*x = WriteStreamOperationStart{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteStreamOperationStart) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteStreamOperationStart) ProtoMessage() {}\n\nfunc (x *WriteStreamOperationStart) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteStreamOperationStart.ProtoReflect.Descriptor instead.\nfunc (*WriteStreamOperationStart) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (x *WriteStreamOperationStart) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetFormat() string {\n\tif x != nil {\n\t\treturn x.Format\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetPartitioningColumnNames() []string {\n\tif x != nil {\n\t\treturn x.PartitioningColumnNames\n\t}\n\treturn nil\n}\n\nfunc (m *WriteStreamOperationStart) GetTrigger() isWriteStreamOperationStart_Trigger {\n\tif m != nil {\n\t\treturn m.Trigger\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetProcessingTimeInterval() string {\n\tif x, ok := x.GetTrigger().(*WriteStreamOperationStart_ProcessingTimeInterval); ok {\n\t\treturn x.ProcessingTimeInterval\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetAvailableNow() bool {\n\tif x, ok := x.GetTrigger().(*WriteStreamOperationStart_AvailableNow); ok {\n\t\treturn x.AvailableNow\n\t}\n\treturn false\n}\n\nfunc (x *WriteStreamOperationStart) GetOnce() bool {\n\tif x, ok := x.GetTrigger().(*WriteStreamOperationStart_Once); ok {\n\t\treturn x.Once\n\t}\n\treturn false\n}\n\nfunc (x *WriteStreamOperationStart) GetContinuousCheckpointInterval() string {\n\tif x, ok := x.GetTrigger().(*WriteStreamOperationStart_ContinuousCheckpointInterval); ok {\n\t\treturn x.ContinuousCheckpointInterval\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetOutputMode() string {\n\tif x != nil {\n\t\treturn x.OutputMode\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetQueryName() string {\n\tif x != nil {\n\t\treturn x.QueryName\n\t}\n\treturn \"\"\n}\n\nfunc (m *WriteStreamOperationStart) GetSinkDestination() isWriteStreamOperationStart_SinkDestination {\n\tif m != nil {\n\t\treturn m.SinkDestination\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetPath() string {\n\tif x, ok := x.GetSinkDestination().(*WriteStreamOperationStart_Path); ok {\n\t\treturn x.Path\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetTableName() string {\n\tif x, ok := x.GetSinkDestination().(*WriteStreamOperationStart_TableName); ok {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStart) GetForeachWriter() *StreamingForeachFunction {\n\tif x != nil {\n\t\treturn x.ForeachWriter\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetForeachBatch() *StreamingForeachFunction {\n\tif x != nil {\n\t\treturn x.ForeachBatch\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStart) GetClusteringColumnNames() []string {\n\tif x != nil {\n\t\treturn x.ClusteringColumnNames\n\t}\n\treturn nil\n}\n\ntype isWriteStreamOperationStart_Trigger interface {\n\tisWriteStreamOperationStart_Trigger()\n}\n\ntype WriteStreamOperationStart_ProcessingTimeInterval struct {\n\tProcessingTimeInterval string `protobuf:\"bytes,5,opt,name=processing_time_interval,json=processingTimeInterval,proto3,oneof\"`\n}\n\ntype WriteStreamOperationStart_AvailableNow struct {\n\tAvailableNow bool `protobuf:\"varint,6,opt,name=available_now,json=availableNow,proto3,oneof\"`\n}\n\ntype WriteStreamOperationStart_Once struct {\n\tOnce bool `protobuf:\"varint,7,opt,name=once,proto3,oneof\"`\n}\n\ntype WriteStreamOperationStart_ContinuousCheckpointInterval struct {\n\tContinuousCheckpointInterval string `protobuf:\"bytes,8,opt,name=continuous_checkpoint_interval,json=continuousCheckpointInterval,proto3,oneof\"`\n}\n\nfunc (*WriteStreamOperationStart_ProcessingTimeInterval) isWriteStreamOperationStart_Trigger() {}\n\nfunc (*WriteStreamOperationStart_AvailableNow) isWriteStreamOperationStart_Trigger() {}\n\nfunc (*WriteStreamOperationStart_Once) isWriteStreamOperationStart_Trigger() {}\n\nfunc (*WriteStreamOperationStart_ContinuousCheckpointInterval) isWriteStreamOperationStart_Trigger() {\n}\n\ntype isWriteStreamOperationStart_SinkDestination interface {\n\tisWriteStreamOperationStart_SinkDestination()\n}\n\ntype WriteStreamOperationStart_Path struct {\n\tPath string `protobuf:\"bytes,11,opt,name=path,proto3,oneof\"`\n}\n\ntype WriteStreamOperationStart_TableName struct {\n\tTableName string `protobuf:\"bytes,12,opt,name=table_name,json=tableName,proto3,oneof\"`\n}\n\nfunc (*WriteStreamOperationStart_Path) isWriteStreamOperationStart_SinkDestination() {}\n\nfunc (*WriteStreamOperationStart_TableName) isWriteStreamOperationStart_SinkDestination() {}\n\ntype StreamingForeachFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Function:\n\t//\n\t//\t*StreamingForeachFunction_PythonFunction\n\t//\t*StreamingForeachFunction_ScalaFunction\n\tFunction isStreamingForeachFunction_Function `protobuf_oneof:\"function\"`\n}\n\nfunc (x *StreamingForeachFunction) Reset() {\n\t*x = StreamingForeachFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingForeachFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingForeachFunction) ProtoMessage() {}\n\nfunc (x *StreamingForeachFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingForeachFunction.ProtoReflect.Descriptor instead.\nfunc (*StreamingForeachFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (m *StreamingForeachFunction) GetFunction() isStreamingForeachFunction_Function {\n\tif m != nil {\n\t\treturn m.Function\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingForeachFunction) GetPythonFunction() *PythonUDF {\n\tif x, ok := x.GetFunction().(*StreamingForeachFunction_PythonFunction); ok {\n\t\treturn x.PythonFunction\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingForeachFunction) GetScalaFunction() *ScalarScalaUDF {\n\tif x, ok := x.GetFunction().(*StreamingForeachFunction_ScalaFunction); ok {\n\t\treturn x.ScalaFunction\n\t}\n\treturn nil\n}\n\ntype isStreamingForeachFunction_Function interface {\n\tisStreamingForeachFunction_Function()\n}\n\ntype StreamingForeachFunction_PythonFunction struct {\n\tPythonFunction *PythonUDF `protobuf:\"bytes,1,opt,name=python_function,json=pythonFunction,proto3,oneof\"`\n}\n\ntype StreamingForeachFunction_ScalaFunction struct {\n\tScalaFunction *ScalarScalaUDF `protobuf:\"bytes,2,opt,name=scala_function,json=scalaFunction,proto3,oneof\"`\n}\n\nfunc (*StreamingForeachFunction_PythonFunction) isStreamingForeachFunction_Function() {}\n\nfunc (*StreamingForeachFunction_ScalaFunction) isStreamingForeachFunction_Function() {}\n\ntype WriteStreamOperationStartResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Query instance. See `StreamingQueryInstanceId`.\n\tQueryId *StreamingQueryInstanceId `protobuf:\"bytes,1,opt,name=query_id,json=queryId,proto3\" json:\"query_id,omitempty\"`\n\t// An optional query name.\n\tName string `protobuf:\"bytes,2,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// Optional query started event if there is any listener registered on the client side.\n\tQueryStartedEventJson *string `protobuf:\"bytes,3,opt,name=query_started_event_json,json=queryStartedEventJson,proto3,oneof\" json:\"query_started_event_json,omitempty\"`\n}\n\nfunc (x *WriteStreamOperationStartResult) Reset() {\n\t*x = WriteStreamOperationStartResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteStreamOperationStartResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteStreamOperationStartResult) ProtoMessage() {}\n\nfunc (x *WriteStreamOperationStartResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteStreamOperationStartResult.ProtoReflect.Descriptor instead.\nfunc (*WriteStreamOperationStartResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (x *WriteStreamOperationStartResult) GetQueryId() *StreamingQueryInstanceId {\n\tif x != nil {\n\t\treturn x.QueryId\n\t}\n\treturn nil\n}\n\nfunc (x *WriteStreamOperationStartResult) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteStreamOperationStartResult) GetQueryStartedEventJson() string {\n\tif x != nil && x.QueryStartedEventJson != nil {\n\t\treturn *x.QueryStartedEventJson\n\t}\n\treturn \"\"\n}\n\n// A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that\n// persists across the streaming runs and `run_id` that changes between each run of the\n// streaming query that resumes from the checkpoint.\ntype StreamingQueryInstanceId struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The unique id of this query that persists across restarts from checkpoint data.\n\t// That is, this id is generated when a query is started for the first time, and\n\t// will be the same every time it is restarted from checkpoint data.\n\tId string `protobuf:\"bytes,1,opt,name=id,proto3\" json:\"id,omitempty\"`\n\t// (Required) The unique id of this run of the query. That is, every start/restart of a query\n\t// will generate a unique run_id. Therefore, every time a query is restarted from\n\t// checkpoint, it will have the same `id` but different `run_id`s.\n\tRunId string `protobuf:\"bytes,2,opt,name=run_id,json=runId,proto3\" json:\"run_id,omitempty\"`\n}\n\nfunc (x *StreamingQueryInstanceId) Reset() {\n\t*x = StreamingQueryInstanceId{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryInstanceId) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryInstanceId) ProtoMessage() {}\n\nfunc (x *StreamingQueryInstanceId) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryInstanceId.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryInstanceId) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *StreamingQueryInstanceId) GetId() string {\n\tif x != nil {\n\t\treturn x.Id\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryInstanceId) GetRunId() string {\n\tif x != nil {\n\t\treturn x.RunId\n\t}\n\treturn \"\"\n}\n\n// Commands for a streaming query.\ntype StreamingQueryCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Query instance. See `StreamingQueryInstanceId`.\n\tQueryId *StreamingQueryInstanceId `protobuf:\"bytes,1,opt,name=query_id,json=queryId,proto3\" json:\"query_id,omitempty\"`\n\t// See documentation for the corresponding API method in StreamingQuery.\n\t//\n\t// Types that are assignable to Command:\n\t//\n\t//\t*StreamingQueryCommand_Status\n\t//\t*StreamingQueryCommand_LastProgress\n\t//\t*StreamingQueryCommand_RecentProgress\n\t//\t*StreamingQueryCommand_Stop\n\t//\t*StreamingQueryCommand_ProcessAllAvailable\n\t//\t*StreamingQueryCommand_Explain\n\t//\t*StreamingQueryCommand_Exception\n\t//\t*StreamingQueryCommand_AwaitTermination\n\tCommand isStreamingQueryCommand_Command `protobuf_oneof:\"command\"`\n}\n\nfunc (x *StreamingQueryCommand) Reset() {\n\t*x = StreamingQueryCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *StreamingQueryCommand) GetQueryId() *StreamingQueryInstanceId {\n\tif x != nil {\n\t\treturn x.QueryId\n\t}\n\treturn nil\n}\n\nfunc (m *StreamingQueryCommand) GetCommand() isStreamingQueryCommand_Command {\n\tif m != nil {\n\t\treturn m.Command\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommand) GetStatus() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_Status); ok {\n\t\treturn x.Status\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetLastProgress() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_LastProgress); ok {\n\t\treturn x.LastProgress\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetRecentProgress() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_RecentProgress); ok {\n\t\treturn x.RecentProgress\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetStop() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_Stop); ok {\n\t\treturn x.Stop\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetProcessAllAvailable() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_ProcessAllAvailable); ok {\n\t\treturn x.ProcessAllAvailable\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetExplain() *StreamingQueryCommand_ExplainCommand {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_Explain); ok {\n\t\treturn x.Explain\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommand) GetException() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_Exception); ok {\n\t\treturn x.Exception\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommand) GetAwaitTermination() *StreamingQueryCommand_AwaitTerminationCommand {\n\tif x, ok := x.GetCommand().(*StreamingQueryCommand_AwaitTermination); ok {\n\t\treturn x.AwaitTermination\n\t}\n\treturn nil\n}\n\ntype isStreamingQueryCommand_Command interface {\n\tisStreamingQueryCommand_Command()\n}\n\ntype StreamingQueryCommand_Status struct {\n\t// status() API.\n\tStatus bool `protobuf:\"varint,2,opt,name=status,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_LastProgress struct {\n\t// lastProgress() API.\n\tLastProgress bool `protobuf:\"varint,3,opt,name=last_progress,json=lastProgress,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_RecentProgress struct {\n\t// recentProgress() API.\n\tRecentProgress bool `protobuf:\"varint,4,opt,name=recent_progress,json=recentProgress,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_Stop struct {\n\t// stop() API. Stops the query.\n\tStop bool `protobuf:\"varint,5,opt,name=stop,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_ProcessAllAvailable struct {\n\t// processAllAvailable() API. Waits till all the available data is processed\n\tProcessAllAvailable bool `protobuf:\"varint,6,opt,name=process_all_available,json=processAllAvailable,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_Explain struct {\n\t// explain() API. Returns logical and physical plans.\n\tExplain *StreamingQueryCommand_ExplainCommand `protobuf:\"bytes,7,opt,name=explain,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_Exception struct {\n\t// exception() API. Returns the exception in the query if any.\n\tException bool `protobuf:\"varint,8,opt,name=exception,proto3,oneof\"`\n}\n\ntype StreamingQueryCommand_AwaitTermination struct {\n\t// awaitTermination() API. Waits for the termination of the query.\n\tAwaitTermination *StreamingQueryCommand_AwaitTerminationCommand `protobuf:\"bytes,9,opt,name=await_termination,json=awaitTermination,proto3,oneof\"`\n}\n\nfunc (*StreamingQueryCommand_Status) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_LastProgress) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_RecentProgress) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_Stop) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_ProcessAllAvailable) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_Explain) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_Exception) isStreamingQueryCommand_Command() {}\n\nfunc (*StreamingQueryCommand_AwaitTermination) isStreamingQueryCommand_Command() {}\n\n// Response for commands on a streaming query.\ntype StreamingQueryCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Query instance id. See `StreamingQueryInstanceId`.\n\tQueryId *StreamingQueryInstanceId `protobuf:\"bytes,1,opt,name=query_id,json=queryId,proto3\" json:\"query_id,omitempty\"`\n\t// Types that are assignable to ResultType:\n\t//\n\t//\t*StreamingQueryCommandResult_Status\n\t//\t*StreamingQueryCommandResult_RecentProgress\n\t//\t*StreamingQueryCommandResult_Explain\n\t//\t*StreamingQueryCommandResult_Exception\n\t//\t*StreamingQueryCommandResult_AwaitTermination\n\tResultType isStreamingQueryCommandResult_ResultType `protobuf_oneof:\"result_type\"`\n}\n\nfunc (x *StreamingQueryCommandResult) Reset() {\n\t*x = StreamingQueryCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *StreamingQueryCommandResult) GetQueryId() *StreamingQueryInstanceId {\n\tif x != nil {\n\t\treturn x.QueryId\n\t}\n\treturn nil\n}\n\nfunc (m *StreamingQueryCommandResult) GetResultType() isStreamingQueryCommandResult_ResultType {\n\tif m != nil {\n\t\treturn m.ResultType\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommandResult) GetStatus() *StreamingQueryCommandResult_StatusResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryCommandResult_Status); ok {\n\t\treturn x.Status\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommandResult) GetRecentProgress() *StreamingQueryCommandResult_RecentProgressResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryCommandResult_RecentProgress); ok {\n\t\treturn x.RecentProgress\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommandResult) GetExplain() *StreamingQueryCommandResult_ExplainResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryCommandResult_Explain); ok {\n\t\treturn x.Explain\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommandResult) GetException() *StreamingQueryCommandResult_ExceptionResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryCommandResult_Exception); ok {\n\t\treturn x.Exception\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryCommandResult) GetAwaitTermination() *StreamingQueryCommandResult_AwaitTerminationResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryCommandResult_AwaitTermination); ok {\n\t\treturn x.AwaitTermination\n\t}\n\treturn nil\n}\n\ntype isStreamingQueryCommandResult_ResultType interface {\n\tisStreamingQueryCommandResult_ResultType()\n}\n\ntype StreamingQueryCommandResult_Status struct {\n\tStatus *StreamingQueryCommandResult_StatusResult `protobuf:\"bytes,2,opt,name=status,proto3,oneof\"`\n}\n\ntype StreamingQueryCommandResult_RecentProgress struct {\n\tRecentProgress *StreamingQueryCommandResult_RecentProgressResult `protobuf:\"bytes,3,opt,name=recent_progress,json=recentProgress,proto3,oneof\"`\n}\n\ntype StreamingQueryCommandResult_Explain struct {\n\tExplain *StreamingQueryCommandResult_ExplainResult `protobuf:\"bytes,4,opt,name=explain,proto3,oneof\"`\n}\n\ntype StreamingQueryCommandResult_Exception struct {\n\tException *StreamingQueryCommandResult_ExceptionResult `protobuf:\"bytes,5,opt,name=exception,proto3,oneof\"`\n}\n\ntype StreamingQueryCommandResult_AwaitTermination struct {\n\tAwaitTermination *StreamingQueryCommandResult_AwaitTerminationResult `protobuf:\"bytes,6,opt,name=await_termination,json=awaitTermination,proto3,oneof\"`\n}\n\nfunc (*StreamingQueryCommandResult_Status) isStreamingQueryCommandResult_ResultType() {}\n\nfunc (*StreamingQueryCommandResult_RecentProgress) isStreamingQueryCommandResult_ResultType() {}\n\nfunc (*StreamingQueryCommandResult_Explain) isStreamingQueryCommandResult_ResultType() {}\n\nfunc (*StreamingQueryCommandResult_Exception) isStreamingQueryCommandResult_ResultType() {}\n\nfunc (*StreamingQueryCommandResult_AwaitTermination) isStreamingQueryCommandResult_ResultType() {}\n\n// Commands for the streaming query manager.\ntype StreamingQueryManagerCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// See documentation for the corresponding API method in StreamingQueryManager.\n\t//\n\t// Types that are assignable to Command:\n\t//\n\t//\t*StreamingQueryManagerCommand_Active\n\t//\t*StreamingQueryManagerCommand_GetQuery\n\t//\t*StreamingQueryManagerCommand_AwaitAnyTermination\n\t//\t*StreamingQueryManagerCommand_ResetTerminated\n\t//\t*StreamingQueryManagerCommand_AddListener\n\t//\t*StreamingQueryManagerCommand_RemoveListener\n\t//\t*StreamingQueryManagerCommand_ListListeners\n\tCommand isStreamingQueryManagerCommand_Command `protobuf_oneof:\"command\"`\n}\n\nfunc (x *StreamingQueryManagerCommand) Reset() {\n\t*x = StreamingQueryManagerCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{11}\n}\n\nfunc (m *StreamingQueryManagerCommand) GetCommand() isStreamingQueryManagerCommand_Command {\n\tif m != nil {\n\t\treturn m.Command\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand) GetActive() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_Active); ok {\n\t\treturn x.Active\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryManagerCommand) GetGetQuery() string {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_GetQuery); ok {\n\t\treturn x.GetQuery\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryManagerCommand) GetAwaitAnyTermination() *StreamingQueryManagerCommand_AwaitAnyTerminationCommand {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_AwaitAnyTermination); ok {\n\t\treturn x.AwaitAnyTermination\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand) GetResetTerminated() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_ResetTerminated); ok {\n\t\treturn x.ResetTerminated\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryManagerCommand) GetAddListener() *StreamingQueryManagerCommand_StreamingQueryListenerCommand {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_AddListener); ok {\n\t\treturn x.AddListener\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand) GetRemoveListener() *StreamingQueryManagerCommand_StreamingQueryListenerCommand {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_RemoveListener); ok {\n\t\treturn x.RemoveListener\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand) GetListListeners() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryManagerCommand_ListListeners); ok {\n\t\treturn x.ListListeners\n\t}\n\treturn false\n}\n\ntype isStreamingQueryManagerCommand_Command interface {\n\tisStreamingQueryManagerCommand_Command()\n}\n\ntype StreamingQueryManagerCommand_Active struct {\n\t// active() API, returns a list of active queries.\n\tActive bool `protobuf:\"varint,1,opt,name=active,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_GetQuery struct {\n\t// get() API, returns the StreamingQuery identified by id.\n\tGetQuery string `protobuf:\"bytes,2,opt,name=get_query,json=getQuery,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_AwaitAnyTermination struct {\n\t// awaitAnyTermination() API, wait until any query terminates or timeout.\n\tAwaitAnyTermination *StreamingQueryManagerCommand_AwaitAnyTerminationCommand `protobuf:\"bytes,3,opt,name=await_any_termination,json=awaitAnyTermination,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_ResetTerminated struct {\n\t// resetTerminated() API.\n\tResetTerminated bool `protobuf:\"varint,4,opt,name=reset_terminated,json=resetTerminated,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_AddListener struct {\n\t// addListener API.\n\tAddListener *StreamingQueryManagerCommand_StreamingQueryListenerCommand `protobuf:\"bytes,5,opt,name=add_listener,json=addListener,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_RemoveListener struct {\n\t// removeListener API.\n\tRemoveListener *StreamingQueryManagerCommand_StreamingQueryListenerCommand `protobuf:\"bytes,6,opt,name=remove_listener,json=removeListener,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommand_ListListeners struct {\n\t// listListeners() API, returns a list of streaming query listeners.\n\tListListeners bool `protobuf:\"varint,7,opt,name=list_listeners,json=listListeners,proto3,oneof\"`\n}\n\nfunc (*StreamingQueryManagerCommand_Active) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_GetQuery) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_AwaitAnyTermination) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_ResetTerminated) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_AddListener) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_RemoveListener) isStreamingQueryManagerCommand_Command() {}\n\nfunc (*StreamingQueryManagerCommand_ListListeners) isStreamingQueryManagerCommand_Command() {}\n\n// Response for commands on the streaming query manager.\ntype StreamingQueryManagerCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ResultType:\n\t//\n\t//\t*StreamingQueryManagerCommandResult_Active\n\t//\t*StreamingQueryManagerCommandResult_Query\n\t//\t*StreamingQueryManagerCommandResult_AwaitAnyTermination\n\t//\t*StreamingQueryManagerCommandResult_ResetTerminated\n\t//\t*StreamingQueryManagerCommandResult_AddListener\n\t//\t*StreamingQueryManagerCommandResult_RemoveListener\n\t//\t*StreamingQueryManagerCommandResult_ListListeners\n\tResultType isStreamingQueryManagerCommandResult_ResultType `protobuf_oneof:\"result_type\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult) Reset() {\n\t*x = StreamingQueryManagerCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12}\n}\n\nfunc (m *StreamingQueryManagerCommandResult) GetResultType() isStreamingQueryManagerCommandResult_ResultType {\n\tif m != nil {\n\t\treturn m.ResultType\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetActive() *StreamingQueryManagerCommandResult_ActiveResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_Active); ok {\n\t\treturn x.Active\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetQuery() *StreamingQueryManagerCommandResult_StreamingQueryInstance {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_Query); ok {\n\t\treturn x.Query\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetAwaitAnyTermination() *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_AwaitAnyTermination); ok {\n\t\treturn x.AwaitAnyTermination\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetResetTerminated() bool {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_ResetTerminated); ok {\n\t\treturn x.ResetTerminated\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetAddListener() bool {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_AddListener); ok {\n\t\treturn x.AddListener\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetRemoveListener() bool {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_RemoveListener); ok {\n\t\treturn x.RemoveListener\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryManagerCommandResult) GetListListeners() *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult {\n\tif x, ok := x.GetResultType().(*StreamingQueryManagerCommandResult_ListListeners); ok {\n\t\treturn x.ListListeners\n\t}\n\treturn nil\n}\n\ntype isStreamingQueryManagerCommandResult_ResultType interface {\n\tisStreamingQueryManagerCommandResult_ResultType()\n}\n\ntype StreamingQueryManagerCommandResult_Active struct {\n\tActive *StreamingQueryManagerCommandResult_ActiveResult `protobuf:\"bytes,1,opt,name=active,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_Query struct {\n\tQuery *StreamingQueryManagerCommandResult_StreamingQueryInstance `protobuf:\"bytes,2,opt,name=query,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_AwaitAnyTermination struct {\n\tAwaitAnyTermination *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult `protobuf:\"bytes,3,opt,name=await_any_termination,json=awaitAnyTermination,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_ResetTerminated struct {\n\tResetTerminated bool `protobuf:\"varint,4,opt,name=reset_terminated,json=resetTerminated,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_AddListener struct {\n\tAddListener bool `protobuf:\"varint,5,opt,name=add_listener,json=addListener,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_RemoveListener struct {\n\tRemoveListener bool `protobuf:\"varint,6,opt,name=remove_listener,json=removeListener,proto3,oneof\"`\n}\n\ntype StreamingQueryManagerCommandResult_ListListeners struct {\n\tListListeners *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult `protobuf:\"bytes,7,opt,name=list_listeners,json=listListeners,proto3,oneof\"`\n}\n\nfunc (*StreamingQueryManagerCommandResult_Active) isStreamingQueryManagerCommandResult_ResultType() {}\n\nfunc (*StreamingQueryManagerCommandResult_Query) isStreamingQueryManagerCommandResult_ResultType() {}\n\nfunc (*StreamingQueryManagerCommandResult_AwaitAnyTermination) isStreamingQueryManagerCommandResult_ResultType() {\n}\n\nfunc (*StreamingQueryManagerCommandResult_ResetTerminated) isStreamingQueryManagerCommandResult_ResultType() {\n}\n\nfunc (*StreamingQueryManagerCommandResult_AddListener) isStreamingQueryManagerCommandResult_ResultType() {\n}\n\nfunc (*StreamingQueryManagerCommandResult_RemoveListener) isStreamingQueryManagerCommandResult_ResultType() {\n}\n\nfunc (*StreamingQueryManagerCommandResult_ListListeners) isStreamingQueryManagerCommandResult_ResultType() {\n}\n\n// The protocol for client-side StreamingQueryListener.\n// This command will only be set when either the first listener is added to the client, or the last\n// listener is removed from the client.\n// The add_listener_bus_listener command will only be set true in the first case.\n// The remove_listener_bus_listener command will only be set true in the second case.\ntype StreamingQueryListenerBusCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Command:\n\t//\n\t//\t*StreamingQueryListenerBusCommand_AddListenerBusListener\n\t//\t*StreamingQueryListenerBusCommand_RemoveListenerBusListener\n\tCommand isStreamingQueryListenerBusCommand_Command `protobuf_oneof:\"command\"`\n}\n\nfunc (x *StreamingQueryListenerBusCommand) Reset() {\n\t*x = StreamingQueryListenerBusCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryListenerBusCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryListenerBusCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryListenerBusCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryListenerBusCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryListenerBusCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{13}\n}\n\nfunc (m *StreamingQueryListenerBusCommand) GetCommand() isStreamingQueryListenerBusCommand_Command {\n\tif m != nil {\n\t\treturn m.Command\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryListenerBusCommand) GetAddListenerBusListener() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryListenerBusCommand_AddListenerBusListener); ok {\n\t\treturn x.AddListenerBusListener\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryListenerBusCommand) GetRemoveListenerBusListener() bool {\n\tif x, ok := x.GetCommand().(*StreamingQueryListenerBusCommand_RemoveListenerBusListener); ok {\n\t\treturn x.RemoveListenerBusListener\n\t}\n\treturn false\n}\n\ntype isStreamingQueryListenerBusCommand_Command interface {\n\tisStreamingQueryListenerBusCommand_Command()\n}\n\ntype StreamingQueryListenerBusCommand_AddListenerBusListener struct {\n\tAddListenerBusListener bool `protobuf:\"varint,1,opt,name=add_listener_bus_listener,json=addListenerBusListener,proto3,oneof\"`\n}\n\ntype StreamingQueryListenerBusCommand_RemoveListenerBusListener struct {\n\tRemoveListenerBusListener bool `protobuf:\"varint,2,opt,name=remove_listener_bus_listener,json=removeListenerBusListener,proto3,oneof\"`\n}\n\nfunc (*StreamingQueryListenerBusCommand_AddListenerBusListener) isStreamingQueryListenerBusCommand_Command() {\n}\n\nfunc (*StreamingQueryListenerBusCommand_RemoveListenerBusListener) isStreamingQueryListenerBusCommand_Command() {\n}\n\n// The protocol for the returned events in the long-running response channel.\ntype StreamingQueryListenerEvent struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The json serialized event, all StreamingQueryListener events have a json method\n\tEventJson string `protobuf:\"bytes,1,opt,name=event_json,json=eventJson,proto3\" json:\"event_json,omitempty\"`\n\t// (Required) Query event type used by client to decide how to deserialize the event_json\n\tEventType StreamingQueryEventType `protobuf:\"varint,2,opt,name=event_type,json=eventType,proto3,enum=spark.connect.StreamingQueryEventType\" json:\"event_type,omitempty\"`\n}\n\nfunc (x *StreamingQueryListenerEvent) Reset() {\n\t*x = StreamingQueryListenerEvent{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryListenerEvent) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryListenerEvent) ProtoMessage() {}\n\nfunc (x *StreamingQueryListenerEvent) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryListenerEvent.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryListenerEvent) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{14}\n}\n\nfunc (x *StreamingQueryListenerEvent) GetEventJson() string {\n\tif x != nil {\n\t\treturn x.EventJson\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryListenerEvent) GetEventType() StreamingQueryEventType {\n\tif x != nil {\n\t\treturn x.EventType\n\t}\n\treturn StreamingQueryEventType_QUERY_PROGRESS_UNSPECIFIED\n}\n\ntype StreamingQueryListenerEventsResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tEvents                   []*StreamingQueryListenerEvent `protobuf:\"bytes,1,rep,name=events,proto3\" json:\"events,omitempty\"`\n\tListenerBusListenerAdded *bool                          `protobuf:\"varint,2,opt,name=listener_bus_listener_added,json=listenerBusListenerAdded,proto3,oneof\" json:\"listener_bus_listener_added,omitempty\"`\n}\n\nfunc (x *StreamingQueryListenerEventsResult) Reset() {\n\t*x = StreamingQueryListenerEventsResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryListenerEventsResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryListenerEventsResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryListenerEventsResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryListenerEventsResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryListenerEventsResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{15}\n}\n\nfunc (x *StreamingQueryListenerEventsResult) GetEvents() []*StreamingQueryListenerEvent {\n\tif x != nil {\n\t\treturn x.Events\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryListenerEventsResult) GetListenerBusListenerAdded() bool {\n\tif x != nil && x.ListenerBusListenerAdded != nil {\n\t\treturn *x.ListenerBusListenerAdded\n\t}\n\treturn false\n}\n\n// Command to get the output of 'SparkContext.resources'\ntype GetResourcesCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *GetResourcesCommand) Reset() {\n\t*x = GetResourcesCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GetResourcesCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GetResourcesCommand) ProtoMessage() {}\n\nfunc (x *GetResourcesCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GetResourcesCommand.ProtoReflect.Descriptor instead.\nfunc (*GetResourcesCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{16}\n}\n\n// Response for command 'GetResourcesCommand'.\ntype GetResourcesCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tResources map[string]*ResourceInformation `protobuf:\"bytes,1,rep,name=resources,proto3\" json:\"resources,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *GetResourcesCommandResult) Reset() {\n\t*x = GetResourcesCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GetResourcesCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GetResourcesCommandResult) ProtoMessage() {}\n\nfunc (x *GetResourcesCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GetResourcesCommandResult.ProtoReflect.Descriptor instead.\nfunc (*GetResourcesCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{17}\n}\n\nfunc (x *GetResourcesCommandResult) GetResources() map[string]*ResourceInformation {\n\tif x != nil {\n\t\treturn x.Resources\n\t}\n\treturn nil\n}\n\n// Command to create ResourceProfile\ntype CreateResourceProfileCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The ResourceProfile to be built on the server-side.\n\tProfile *ResourceProfile `protobuf:\"bytes,1,opt,name=profile,proto3\" json:\"profile,omitempty\"`\n}\n\nfunc (x *CreateResourceProfileCommand) Reset() {\n\t*x = CreateResourceProfileCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CreateResourceProfileCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CreateResourceProfileCommand) ProtoMessage() {}\n\nfunc (x *CreateResourceProfileCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CreateResourceProfileCommand.ProtoReflect.Descriptor instead.\nfunc (*CreateResourceProfileCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{18}\n}\n\nfunc (x *CreateResourceProfileCommand) GetProfile() *ResourceProfile {\n\tif x != nil {\n\t\treturn x.Profile\n\t}\n\treturn nil\n}\n\n// Response for command 'CreateResourceProfileCommand'.\ntype CreateResourceProfileCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Server-side generated resource profile id.\n\tProfileId int32 `protobuf:\"varint,1,opt,name=profile_id,json=profileId,proto3\" json:\"profile_id,omitempty\"`\n}\n\nfunc (x *CreateResourceProfileCommandResult) Reset() {\n\t*x = CreateResourceProfileCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CreateResourceProfileCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CreateResourceProfileCommandResult) ProtoMessage() {}\n\nfunc (x *CreateResourceProfileCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CreateResourceProfileCommandResult.ProtoReflect.Descriptor instead.\nfunc (*CreateResourceProfileCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{19}\n}\n\nfunc (x *CreateResourceProfileCommandResult) GetProfileId() int32 {\n\tif x != nil {\n\t\treturn x.ProfileId\n\t}\n\treturn 0\n}\n\n// Command to remove `CashedRemoteRelation`\ntype RemoveCachedRemoteRelationCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The remote to be related\n\tRelation *CachedRemoteRelation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n}\n\nfunc (x *RemoveCachedRemoteRelationCommand) Reset() {\n\t*x = RemoveCachedRemoteRelationCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RemoveCachedRemoteRelationCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RemoveCachedRemoteRelationCommand) ProtoMessage() {}\n\nfunc (x *RemoveCachedRemoteRelationCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RemoveCachedRemoteRelationCommand.ProtoReflect.Descriptor instead.\nfunc (*RemoveCachedRemoteRelationCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{20}\n}\n\nfunc (x *RemoveCachedRemoteRelationCommand) GetRelation() *CachedRemoteRelation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\ntype CheckpointCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The logical plan to checkpoint.\n\tRelation *Relation `protobuf:\"bytes,1,opt,name=relation,proto3\" json:\"relation,omitempty\"`\n\t// (Required) Locally checkpoint using a local temporary\n\t// directory in Spark Connect server (Spark Driver)\n\tLocal bool `protobuf:\"varint,2,opt,name=local,proto3\" json:\"local,omitempty\"`\n\t// (Required) Whether to checkpoint this dataframe immediately.\n\tEager bool `protobuf:\"varint,3,opt,name=eager,proto3\" json:\"eager,omitempty\"`\n\t// (Optional) For local checkpoint, the storage level to use.\n\tStorageLevel *StorageLevel `protobuf:\"bytes,4,opt,name=storage_level,json=storageLevel,proto3,oneof\" json:\"storage_level,omitempty\"`\n}\n\nfunc (x *CheckpointCommand) Reset() {\n\t*x = CheckpointCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CheckpointCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CheckpointCommand) ProtoMessage() {}\n\nfunc (x *CheckpointCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CheckpointCommand.ProtoReflect.Descriptor instead.\nfunc (*CheckpointCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{21}\n}\n\nfunc (x *CheckpointCommand) GetRelation() *Relation {\n\tif x != nil {\n\t\treturn x.Relation\n\t}\n\treturn nil\n}\n\nfunc (x *CheckpointCommand) GetLocal() bool {\n\tif x != nil {\n\t\treturn x.Local\n\t}\n\treturn false\n}\n\nfunc (x *CheckpointCommand) GetEager() bool {\n\tif x != nil {\n\t\treturn x.Eager\n\t}\n\treturn false\n}\n\nfunc (x *CheckpointCommand) GetStorageLevel() *StorageLevel {\n\tif x != nil {\n\t\treturn x.StorageLevel\n\t}\n\treturn nil\n}\n\ntype MergeIntoTableCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The name of the target table.\n\tTargetTableName string `protobuf:\"bytes,1,opt,name=target_table_name,json=targetTableName,proto3\" json:\"target_table_name,omitempty\"`\n\t// (Required) The relation of the source table.\n\tSourceTablePlan *Relation `protobuf:\"bytes,2,opt,name=source_table_plan,json=sourceTablePlan,proto3\" json:\"source_table_plan,omitempty\"`\n\t// (Required) The condition to match the source and target.\n\tMergeCondition *Expression `protobuf:\"bytes,3,opt,name=merge_condition,json=mergeCondition,proto3\" json:\"merge_condition,omitempty\"`\n\t// (Optional) The actions to be taken when the condition is matched.\n\tMatchActions []*Expression `protobuf:\"bytes,4,rep,name=match_actions,json=matchActions,proto3\" json:\"match_actions,omitempty\"`\n\t// (Optional) The actions to be taken when the condition is not matched.\n\tNotMatchedActions []*Expression `protobuf:\"bytes,5,rep,name=not_matched_actions,json=notMatchedActions,proto3\" json:\"not_matched_actions,omitempty\"`\n\t// (Optional) The actions to be taken when the condition is not matched by source.\n\tNotMatchedBySourceActions []*Expression `protobuf:\"bytes,6,rep,name=not_matched_by_source_actions,json=notMatchedBySourceActions,proto3\" json:\"not_matched_by_source_actions,omitempty\"`\n\t// (Required) Whether to enable schema evolution.\n\tWithSchemaEvolution bool `protobuf:\"varint,7,opt,name=with_schema_evolution,json=withSchemaEvolution,proto3\" json:\"with_schema_evolution,omitempty\"`\n}\n\nfunc (x *MergeIntoTableCommand) Reset() {\n\t*x = MergeIntoTableCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MergeIntoTableCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MergeIntoTableCommand) ProtoMessage() {}\n\nfunc (x *MergeIntoTableCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MergeIntoTableCommand.ProtoReflect.Descriptor instead.\nfunc (*MergeIntoTableCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{22}\n}\n\nfunc (x *MergeIntoTableCommand) GetTargetTableName() string {\n\tif x != nil {\n\t\treturn x.TargetTableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *MergeIntoTableCommand) GetSourceTablePlan() *Relation {\n\tif x != nil {\n\t\treturn x.SourceTablePlan\n\t}\n\treturn nil\n}\n\nfunc (x *MergeIntoTableCommand) GetMergeCondition() *Expression {\n\tif x != nil {\n\t\treturn x.MergeCondition\n\t}\n\treturn nil\n}\n\nfunc (x *MergeIntoTableCommand) GetMatchActions() []*Expression {\n\tif x != nil {\n\t\treturn x.MatchActions\n\t}\n\treturn nil\n}\n\nfunc (x *MergeIntoTableCommand) GetNotMatchedActions() []*Expression {\n\tif x != nil {\n\t\treturn x.NotMatchedActions\n\t}\n\treturn nil\n}\n\nfunc (x *MergeIntoTableCommand) GetNotMatchedBySourceActions() []*Expression {\n\tif x != nil {\n\t\treturn x.NotMatchedBySourceActions\n\t}\n\treturn nil\n}\n\nfunc (x *MergeIntoTableCommand) GetWithSchemaEvolution() bool {\n\tif x != nil {\n\t\treturn x.WithSchemaEvolution\n\t}\n\treturn false\n}\n\n// Execute an arbitrary string command inside an external execution engine\ntype ExecuteExternalCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The class name of the runner that implements `ExternalCommandRunner`\n\tRunner string `protobuf:\"bytes,1,opt,name=runner,proto3\" json:\"runner,omitempty\"`\n\t// (Required) The target command to be executed.\n\tCommand string `protobuf:\"bytes,2,opt,name=command,proto3\" json:\"command,omitempty\"`\n\t// (Optional) The options for the runner.\n\tOptions map[string]string `protobuf:\"bytes,3,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *ExecuteExternalCommand) Reset() {\n\t*x = ExecuteExternalCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecuteExternalCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecuteExternalCommand) ProtoMessage() {}\n\nfunc (x *ExecuteExternalCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecuteExternalCommand.ProtoReflect.Descriptor instead.\nfunc (*ExecuteExternalCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{23}\n}\n\nfunc (x *ExecuteExternalCommand) GetRunner() string {\n\tif x != nil {\n\t\treturn x.Runner\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecuteExternalCommand) GetCommand() string {\n\tif x != nil {\n\t\treturn x.Command\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecuteExternalCommand) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\ntype WriteOperation_SaveTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The table name.\n\tTableName string `protobuf:\"bytes,1,opt,name=table_name,json=tableName,proto3\" json:\"table_name,omitempty\"`\n\t// (Required) The method to be called to write to the table.\n\tSaveMethod WriteOperation_SaveTable_TableSaveMethod `protobuf:\"varint,2,opt,name=save_method,json=saveMethod,proto3,enum=spark.connect.WriteOperation_SaveTable_TableSaveMethod\" json:\"save_method,omitempty\"`\n}\n\nfunc (x *WriteOperation_SaveTable) Reset() {\n\t*x = WriteOperation_SaveTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[27]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteOperation_SaveTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteOperation_SaveTable) ProtoMessage() {}\n\nfunc (x *WriteOperation_SaveTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[27]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteOperation_SaveTable.ProtoReflect.Descriptor instead.\nfunc (*WriteOperation_SaveTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{3, 1}\n}\n\nfunc (x *WriteOperation_SaveTable) GetTableName() string {\n\tif x != nil {\n\t\treturn x.TableName\n\t}\n\treturn \"\"\n}\n\nfunc (x *WriteOperation_SaveTable) GetSaveMethod() WriteOperation_SaveTable_TableSaveMethod {\n\tif x != nil {\n\t\treturn x.SaveMethod\n\t}\n\treturn WriteOperation_SaveTable_TABLE_SAVE_METHOD_UNSPECIFIED\n}\n\ntype WriteOperation_BucketBy struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tBucketColumnNames []string `protobuf:\"bytes,1,rep,name=bucket_column_names,json=bucketColumnNames,proto3\" json:\"bucket_column_names,omitempty\"`\n\tNumBuckets        int32    `protobuf:\"varint,2,opt,name=num_buckets,json=numBuckets,proto3\" json:\"num_buckets,omitempty\"`\n}\n\nfunc (x *WriteOperation_BucketBy) Reset() {\n\t*x = WriteOperation_BucketBy{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[28]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WriteOperation_BucketBy) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WriteOperation_BucketBy) ProtoMessage() {}\n\nfunc (x *WriteOperation_BucketBy) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[28]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WriteOperation_BucketBy.ProtoReflect.Descriptor instead.\nfunc (*WriteOperation_BucketBy) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{3, 2}\n}\n\nfunc (x *WriteOperation_BucketBy) GetBucketColumnNames() []string {\n\tif x != nil {\n\t\treturn x.BucketColumnNames\n\t}\n\treturn nil\n}\n\nfunc (x *WriteOperation_BucketBy) GetNumBuckets() int32 {\n\tif x != nil {\n\t\treturn x.NumBuckets\n\t}\n\treturn 0\n}\n\ntype StreamingQueryCommand_ExplainCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// TODO: Consider reusing Explain from AnalyzePlanRequest message.\n\t//\n\t//\tWe can not do this right now since it base.proto imports this file.\n\tExtended bool `protobuf:\"varint,1,opt,name=extended,proto3\" json:\"extended,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommand_ExplainCommand) Reset() {\n\t*x = StreamingQueryCommand_ExplainCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[32]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommand_ExplainCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommand_ExplainCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommand_ExplainCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[32]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommand_ExplainCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommand_ExplainCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{9, 0}\n}\n\nfunc (x *StreamingQueryCommand_ExplainCommand) GetExtended() bool {\n\tif x != nil {\n\t\treturn x.Extended\n\t}\n\treturn false\n}\n\ntype StreamingQueryCommand_AwaitTerminationCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTimeoutMs *int64 `protobuf:\"varint,2,opt,name=timeout_ms,json=timeoutMs,proto3,oneof\" json:\"timeout_ms,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommand_AwaitTerminationCommand) Reset() {\n\t*x = StreamingQueryCommand_AwaitTerminationCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[33]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommand_AwaitTerminationCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommand_AwaitTerminationCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommand_AwaitTerminationCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[33]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommand_AwaitTerminationCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommand_AwaitTerminationCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{9, 1}\n}\n\nfunc (x *StreamingQueryCommand_AwaitTerminationCommand) GetTimeoutMs() int64 {\n\tif x != nil && x.TimeoutMs != nil {\n\t\treturn *x.TimeoutMs\n\t}\n\treturn 0\n}\n\ntype StreamingQueryCommandResult_StatusResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// See documentation for these Scala 'StreamingQueryStatus' struct\n\tStatusMessage   string `protobuf:\"bytes,1,opt,name=status_message,json=statusMessage,proto3\" json:\"status_message,omitempty\"`\n\tIsDataAvailable bool   `protobuf:\"varint,2,opt,name=is_data_available,json=isDataAvailable,proto3\" json:\"is_data_available,omitempty\"`\n\tIsTriggerActive bool   `protobuf:\"varint,3,opt,name=is_trigger_active,json=isTriggerActive,proto3\" json:\"is_trigger_active,omitempty\"`\n\tIsActive        bool   `protobuf:\"varint,4,opt,name=is_active,json=isActive,proto3\" json:\"is_active,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) Reset() {\n\t*x = StreamingQueryCommandResult_StatusResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[34]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult_StatusResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[34]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult_StatusResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult_StatusResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10, 0}\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) GetStatusMessage() string {\n\tif x != nil {\n\t\treturn x.StatusMessage\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) GetIsDataAvailable() bool {\n\tif x != nil {\n\t\treturn x.IsDataAvailable\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) GetIsTriggerActive() bool {\n\tif x != nil {\n\t\treturn x.IsTriggerActive\n\t}\n\treturn false\n}\n\nfunc (x *StreamingQueryCommandResult_StatusResult) GetIsActive() bool {\n\tif x != nil {\n\t\treturn x.IsActive\n\t}\n\treturn false\n}\n\ntype StreamingQueryCommandResult_RecentProgressResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Progress reports as an array of json strings.\n\tRecentProgressJson []string `protobuf:\"bytes,5,rep,name=recent_progress_json,json=recentProgressJson,proto3\" json:\"recent_progress_json,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommandResult_RecentProgressResult) Reset() {\n\t*x = StreamingQueryCommandResult_RecentProgressResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[35]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult_RecentProgressResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult_RecentProgressResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult_RecentProgressResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[35]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult_RecentProgressResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult_RecentProgressResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10, 1}\n}\n\nfunc (x *StreamingQueryCommandResult_RecentProgressResult) GetRecentProgressJson() []string {\n\tif x != nil {\n\t\treturn x.RecentProgressJson\n\t}\n\treturn nil\n}\n\ntype StreamingQueryCommandResult_ExplainResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Logical and physical plans as string\n\tResult string `protobuf:\"bytes,1,opt,name=result,proto3\" json:\"result,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommandResult_ExplainResult) Reset() {\n\t*x = StreamingQueryCommandResult_ExplainResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[36]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult_ExplainResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult_ExplainResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult_ExplainResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[36]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult_ExplainResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult_ExplainResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10, 2}\n}\n\nfunc (x *StreamingQueryCommandResult_ExplainResult) GetResult() string {\n\tif x != nil {\n\t\treturn x.Result\n\t}\n\treturn \"\"\n}\n\ntype StreamingQueryCommandResult_ExceptionResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Exception message as string, maps to the return value of original\n\t// StreamingQueryException's toString method\n\tExceptionMessage *string `protobuf:\"bytes,1,opt,name=exception_message,json=exceptionMessage,proto3,oneof\" json:\"exception_message,omitempty\"`\n\t// (Optional) Exception error class as string\n\tErrorClass *string `protobuf:\"bytes,2,opt,name=error_class,json=errorClass,proto3,oneof\" json:\"error_class,omitempty\"`\n\t// (Optional) Exception stack trace as string\n\tStackTrace *string `protobuf:\"bytes,3,opt,name=stack_trace,json=stackTrace,proto3,oneof\" json:\"stack_trace,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) Reset() {\n\t*x = StreamingQueryCommandResult_ExceptionResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[37]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult_ExceptionResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[37]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult_ExceptionResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult_ExceptionResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10, 3}\n}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) GetExceptionMessage() string {\n\tif x != nil && x.ExceptionMessage != nil {\n\t\treturn *x.ExceptionMessage\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) GetErrorClass() string {\n\tif x != nil && x.ErrorClass != nil {\n\t\treturn *x.ErrorClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *StreamingQueryCommandResult_ExceptionResult) GetStackTrace() string {\n\tif x != nil && x.StackTrace != nil {\n\t\treturn *x.StackTrace\n\t}\n\treturn \"\"\n}\n\ntype StreamingQueryCommandResult_AwaitTerminationResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTerminated bool `protobuf:\"varint,1,opt,name=terminated,proto3\" json:\"terminated,omitempty\"`\n}\n\nfunc (x *StreamingQueryCommandResult_AwaitTerminationResult) Reset() {\n\t*x = StreamingQueryCommandResult_AwaitTerminationResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[38]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryCommandResult_AwaitTerminationResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryCommandResult_AwaitTerminationResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryCommandResult_AwaitTerminationResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[38]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryCommandResult_AwaitTerminationResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryCommandResult_AwaitTerminationResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{10, 4}\n}\n\nfunc (x *StreamingQueryCommandResult_AwaitTerminationResult) GetTerminated() bool {\n\tif x != nil {\n\t\treturn x.Terminated\n\t}\n\treturn false\n}\n\ntype StreamingQueryManagerCommand_AwaitAnyTerminationCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The waiting time in milliseconds to wait for any query to terminate.\n\tTimeoutMs *int64 `protobuf:\"varint,1,opt,name=timeout_ms,json=timeoutMs,proto3,oneof\" json:\"timeout_ms,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Reset() {\n\t*x = StreamingQueryManagerCommand_AwaitAnyTerminationCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[39]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommand_AwaitAnyTerminationCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[39]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommand_AwaitAnyTerminationCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{11, 0}\n}\n\nfunc (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) GetTimeoutMs() int64 {\n\tif x != nil && x.TimeoutMs != nil {\n\t\treturn *x.TimeoutMs\n\t}\n\treturn 0\n}\n\ntype StreamingQueryManagerCommand_StreamingQueryListenerCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tListenerPayload       []byte     `protobuf:\"bytes,1,opt,name=listener_payload,json=listenerPayload,proto3\" json:\"listener_payload,omitempty\"`\n\tPythonListenerPayload *PythonUDF `protobuf:\"bytes,2,opt,name=python_listener_payload,json=pythonListenerPayload,proto3,oneof\" json:\"python_listener_payload,omitempty\"`\n\tId                    string     `protobuf:\"bytes,3,opt,name=id,proto3\" json:\"id,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) Reset() {\n\t*x = StreamingQueryManagerCommand_StreamingQueryListenerCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[40]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommand_StreamingQueryListenerCommand) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[40]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommand_StreamingQueryListenerCommand.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommand_StreamingQueryListenerCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{11, 1}\n}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) GetListenerPayload() []byte {\n\tif x != nil {\n\t\treturn x.ListenerPayload\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) GetPythonListenerPayload() *PythonUDF {\n\tif x != nil {\n\t\treturn x.PythonListenerPayload\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) GetId() string {\n\tif x != nil {\n\t\treturn x.Id\n\t}\n\treturn \"\"\n}\n\ntype StreamingQueryManagerCommandResult_ActiveResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tActiveQueries []*StreamingQueryManagerCommandResult_StreamingQueryInstance `protobuf:\"bytes,1,rep,name=active_queries,json=activeQueries,proto3\" json:\"active_queries,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ActiveResult) Reset() {\n\t*x = StreamingQueryManagerCommandResult_ActiveResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[41]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ActiveResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult_ActiveResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult_ActiveResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[41]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult_ActiveResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult_ActiveResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12, 0}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ActiveResult) GetActiveQueries() []*StreamingQueryManagerCommandResult_StreamingQueryInstance {\n\tif x != nil {\n\t\treturn x.ActiveQueries\n\t}\n\treturn nil\n}\n\ntype StreamingQueryManagerCommandResult_StreamingQueryInstance struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The id and runId of this query.\n\tId *StreamingQueryInstanceId `protobuf:\"bytes,1,opt,name=id,proto3\" json:\"id,omitempty\"`\n\t// (Optional) The name of this query.\n\tName *string `protobuf:\"bytes,2,opt,name=name,proto3,oneof\" json:\"name,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) Reset() {\n\t*x = StreamingQueryManagerCommandResult_StreamingQueryInstance{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[42]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult_StreamingQueryInstance) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[42]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult_StreamingQueryInstance.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult_StreamingQueryInstance) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12, 1}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) GetId() *StreamingQueryInstanceId {\n\tif x != nil {\n\t\treturn x.Id\n\t}\n\treturn nil\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) GetName() string {\n\tif x != nil && x.Name != nil {\n\t\treturn *x.Name\n\t}\n\treturn \"\"\n}\n\ntype StreamingQueryManagerCommandResult_AwaitAnyTerminationResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTerminated bool `protobuf:\"varint,1,opt,name=terminated,proto3\" json:\"terminated,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) Reset() {\n\t*x = StreamingQueryManagerCommandResult_AwaitAnyTerminationResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[43]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[43]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult_AwaitAnyTerminationResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12, 2}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) GetTerminated() bool {\n\tif x != nil {\n\t\treturn x.Terminated\n\t}\n\treturn false\n}\n\ntype StreamingQueryManagerCommandResult_StreamingQueryListenerInstance struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tListenerPayload []byte `protobuf:\"bytes,1,opt,name=listener_payload,json=listenerPayload,proto3\" json:\"listener_payload,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) Reset() {\n\t*x = StreamingQueryManagerCommandResult_StreamingQueryListenerInstance{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[44]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[44]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult_StreamingQueryListenerInstance.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12, 3}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInstance) GetListenerPayload() []byte {\n\tif x != nil {\n\t\treturn x.ListenerPayload\n\t}\n\treturn nil\n}\n\ntype StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Reference IDs of listener instances.\n\tListenerIds []string `protobuf:\"bytes,1,rep,name=listener_ids,json=listenerIds,proto3\" json:\"listener_ids,omitempty\"`\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) Reset() {\n\t*x = StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_commands_proto_msgTypes[45]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) ProtoMessage() {}\n\nfunc (x *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_commands_proto_msgTypes[45]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult.ProtoReflect.Descriptor instead.\nfunc (*StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_commands_proto_rawDescGZIP(), []int{12, 4}\n}\n\nfunc (x *StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult) GetListenerIds() []string {\n\tif x != nil {\n\t\treturn x.ListenerIds\n\t}\n\treturn nil\n}\n\nvar File_spark_connect_commands_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_commands_proto_rawDesc = []byte{\n\t0x0a, 0x1c, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x19, 0x67,\n\t0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61,\n\t0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1a, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70,\n\t0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70,\n\t0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x16, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2f, 0x6d, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x70, 0x69, 0x70, 0x65,\n\t0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xfb, 0x0e, 0x0a, 0x07,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x5d, 0x0a, 0x11, 0x72, 0x65, 0x67, 0x69, 0x73,\n\t0x74, 0x65, 0x72, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55,\n\t0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x48, 0x00, 0x52, 0x10, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x46, 0x75,\n\t0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x48, 0x0a, 0x0f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f,\n\t0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00,\n\t0x52, 0x0e, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x12, 0x5f, 0x0a, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66,\n\t0x72, 0x61, 0x6d, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x56,\n\t0x69, 0x65, 0x77, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x13, 0x63, 0x72,\n\t0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x72, 0x61, 0x6d, 0x65, 0x56, 0x69, 0x65,\n\t0x77, 0x12, 0x4f, 0x0a, 0x12, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x32, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x72,\n\t0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x48, 0x00,\n\t0x52, 0x10, 0x77, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x56, 0x32, 0x12, 0x3c, 0x0a, 0x0b, 0x73, 0x71, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x12, 0x6b, 0x0a, 0x1c, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d,\n\t0x5f, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74,\n\t0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74,\n\t0x48, 0x00, 0x52, 0x19, 0x77, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f,\n\t0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x5e, 0x0a,\n\t0x17, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79,\n\t0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53,\n\t0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x15, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,\n\t0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x58, 0x0a,\n\t0x15, 0x67, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x5f, 0x63,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x47, 0x65, 0x74,\n\t0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x48, 0x00, 0x52, 0x13, 0x67, 0x65, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x74, 0x0a, 0x1f, 0x73, 0x74, 0x72, 0x65, 0x61,\n\t0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,\n\t0x65, 0x72, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d,\n\t0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52,\n\t0x1c, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d,\n\t0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x6d, 0x0a,\n\t0x17, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f,\n\t0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44,\n\t0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x15, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54,\n\t0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x81, 0x01, 0x0a,\n\t0x24, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79,\n\t0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x63, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65,\n\t0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x42, 0x75, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x20,\n\t0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69,\n\t0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x42, 0x75, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x64, 0x61, 0x74,\n\t0x61, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44,\n\t0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65,\n\t0x48, 0x00, 0x52, 0x12, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61,\n\t0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x74, 0x0a, 0x1f, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65,\n\t0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c,\n\t0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72,\n\t0x6f, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x1c,\n\t0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72,\n\t0x6f, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x51, 0x0a, 0x12,\n\t0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f,\n\t0x69, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x11, 0x63, 0x68,\n\t0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12,\n\t0x84, 0x01, 0x0a, 0x25, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65,\n\t0x64, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65, 0x64, 0x52, 0x65, 0x6d, 0x6f,\n\t0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x48, 0x00, 0x52, 0x21, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x61, 0x63, 0x68, 0x65,\n\t0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x5f, 0x0a, 0x18, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f,\n\t0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x49, 0x6e,\n\t0x74, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00,\n\t0x52, 0x15, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x49, 0x6e, 0x74, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x6d, 0x6c, 0x5f, 0x63, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x09, 0x6d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x12, 0x61, 0x0a, 0x18, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x5f, 0x65, 0x78,\n\t0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x12,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65,\n\t0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x16, 0x65,\n\t0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x4b, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,\n\t0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48,\n\t0x00, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x12, 0x35, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18,\n\t0xe7, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x48, 0x00, 0x52, 0x09,\n\t0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x0e, 0x0a, 0x0c, 0x63, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xaa, 0x04, 0x0a, 0x0a, 0x53, 0x71,\n\t0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x14, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x12, 0x3b,\n\t0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x71, 0x6c,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x41, 0x72, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72,\n\t0x79, 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x61, 0x72, 0x67, 0x73, 0x12, 0x40, 0x0a, 0x08, 0x70,\n\t0x6f, 0x73, 0x5f, 0x61, 0x72, 0x67, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x42, 0x02, 0x18, 0x01, 0x52, 0x07, 0x70, 0x6f, 0x73, 0x41, 0x72, 0x67, 0x73, 0x12, 0x5a, 0x0a,\n\t0x0f, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x5f, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73,\n\t0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73,\n\t0x45, 0x6e, 0x74, 0x72, 0x79, 0x42, 0x02, 0x18, 0x01, 0x52, 0x0e, 0x6e, 0x61, 0x6d, 0x65, 0x64,\n\t0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x42, 0x0a, 0x0d, 0x70, 0x6f, 0x73,\n\t0x5f, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x18, 0x01, 0x52,\n\t0x0c, 0x70, 0x6f, 0x73, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2d, 0x0a,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x1a, 0x5a, 0x0a, 0x09,\n\t0x41, 0x72, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x5c, 0x0a, 0x13, 0x4e, 0x61, 0x6d, 0x65,\n\t0x64, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,\n\t0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,\n\t0x79, 0x12, 0x2f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x96, 0x01, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74,\n\t0x65, 0x44, 0x61, 0x74, 0x61, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x56, 0x69, 0x65, 0x77, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x67,\n\t0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x47,\n\t0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65,\n\t0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x22,\n\t0xca, 0x08, 0x0a, 0x0e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75,\n\t0x74, 0x12, 0x1b, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x01, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x88, 0x01, 0x01, 0x12, 0x14,\n\t0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x04,\n\t0x70, 0x61, 0x74, 0x68, 0x12, 0x3f, 0x0a, 0x05, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x04, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x05,\n\t0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x3a, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20,\n\t0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6d, 0x6f, 0x64,\n\t0x65, 0x12, 0x2a, 0x0a, 0x11, 0x73, 0x6f, 0x72, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x73, 0x6f,\n\t0x72, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x31, 0x0a,\n\t0x14, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f,\n\t0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x70, 0x61, 0x72,\n\t0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,\n\t0x12, 0x43, 0x0a, 0x09, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x5f, 0x62, 0x79, 0x18, 0x08, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x79, 0x52, 0x08, 0x62, 0x75, 0x63,\n\t0x6b, 0x65, 0x74, 0x42, 0x79, 0x12, 0x44, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74,\n\t0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x63,\n\t0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72,\n\t0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,\n\t0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x82, 0x02, 0x0a, 0x09, 0x53, 0x61, 0x76, 0x65, 0x54,\n\t0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e,\n\t0x61, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0b, 0x73, 0x61, 0x76, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x68,\n\t0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70,\n\t0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x54, 0x61, 0x62, 0x6c,\n\t0x65, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x61, 0x76, 0x65, 0x4d, 0x65, 0x74, 0x68, 0x6f,\n\t0x64, 0x52, 0x0a, 0x73, 0x61, 0x76, 0x65, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x22, 0x7c, 0x0a,\n\t0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x61, 0x76, 0x65, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64,\n\t0x12, 0x21, 0x0a, 0x1d, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d,\n\t0x45, 0x54, 0x48, 0x4f, 0x44, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45,\n\t0x44, 0x10, 0x00, 0x12, 0x23, 0x0a, 0x1f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x53, 0x41, 0x56,\n\t0x45, 0x5f, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x41, 0x53,\n\t0x5f, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x54, 0x41, 0x42, 0x4c,\n\t0x45, 0x5f, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d, 0x45, 0x54, 0x48, 0x4f, 0x44, 0x5f, 0x49, 0x4e,\n\t0x53, 0x45, 0x52, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x4f, 0x10, 0x02, 0x1a, 0x5b, 0x0a, 0x08, 0x42,\n\t0x75, 0x63, 0x6b, 0x65, 0x74, 0x42, 0x79, 0x12, 0x2e, 0x0a, 0x13, 0x62, 0x75, 0x63, 0x6b, 0x65,\n\t0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x01,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x62, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x62,\n\t0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x6e, 0x75,\n\t0x6d, 0x42, 0x75, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x22, 0x89, 0x01, 0x0a, 0x08, 0x53, 0x61, 0x76,\n\t0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d, 0x4f,\n\t0x44, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00,\n\t0x12, 0x14, 0x0a, 0x10, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50,\n\t0x50, 0x45, 0x4e, 0x44, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d,\n\t0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x02, 0x12,\n\t0x1d, 0x0a, 0x19, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x45, 0x52, 0x52,\n\t0x4f, 0x52, 0x5f, 0x49, 0x46, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x10, 0x03, 0x12, 0x14,\n\t0x0a, 0x10, 0x53, 0x41, 0x56, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, 0x4f,\n\t0x52, 0x45, 0x10, 0x04, 0x42, 0x0b, 0x0a, 0x09, 0x73, 0x61, 0x76, 0x65, 0x5f, 0x74, 0x79, 0x70,\n\t0x65, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0xdc, 0x06, 0x0a,\n\t0x10, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56,\n\t0x32, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12,\n\t0x1f, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x00, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x88, 0x01, 0x01,\n\t0x12, 0x4c, 0x0a, 0x14, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67,\n\t0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x70, 0x61, 0x72, 0x74, 0x69,\n\t0x74, 0x69, 0x6f, 0x6e, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x46,\n\t0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32,\n\t0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f,\n\t0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x5f, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f,\n\t0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x34, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56,\n\t0x32, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,\n\t0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x72, 0x6f,\n\t0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x04, 0x6d, 0x6f, 0x64, 0x65, 0x18,\n\t0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x2e, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6d, 0x6f, 0x64,\n\t0x65, 0x12, 0x4a, 0x0a, 0x13, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x63,\n\t0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x6f, 0x76, 0x65, 0x72, 0x77,\n\t0x72, 0x69, 0x74, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a,\n\t0x12, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6c, 0x75, 0x73, 0x74,\n\t0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c,\n\t0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03,\n\t0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14,\n\t0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x54, 0x61, 0x62, 0x6c,\n\t0x65, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79,\n\t0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b,\n\t0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9f, 0x01, 0x0a,\n\t0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e,\n\t0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x4d,\n\t0x4f, 0x44, 0x45, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e,\n\t0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x02,\n\t0x12, 0x1d, 0x0a, 0x19, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x56, 0x45, 0x52, 0x57, 0x52, 0x49,\n\t0x54, 0x45, 0x5f, 0x50, 0x41, 0x52, 0x54, 0x49, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x03, 0x12,\n\t0x0f, 0x0a, 0x0b, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x41, 0x50, 0x50, 0x45, 0x4e, 0x44, 0x10, 0x04,\n\t0x12, 0x10, 0x0a, 0x0c, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45,\n\t0x10, 0x05, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x52, 0x45, 0x41, 0x54,\n\t0x45, 0x5f, 0x4f, 0x52, 0x5f, 0x52, 0x45, 0x50, 0x4c, 0x41, 0x43, 0x45, 0x10, 0x06, 0x42, 0x0b,\n\t0x0a, 0x09, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x64, 0x65, 0x72, 0x22, 0xd8, 0x06, 0x0a, 0x19,\n\t0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x65, 0x72, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70,\n\t0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d,\n\t0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74,\n\t0x12, 0x4f, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x35, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x2e, 0x4f, 0x70, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,\n\t0x73, 0x12, 0x3a, 0x0a, 0x19, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x69, 0x6e,\n\t0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x04,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x17, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x69,\n\t0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x3a, 0x0a,\n\t0x18, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65,\n\t0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x16, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d,\n\t0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x25, 0x0a, 0x0d, 0x61, 0x76, 0x61,\n\t0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x77, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08,\n\t0x48, 0x00, 0x52, 0x0c, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x77,\n\t0x12, 0x14, 0x0a, 0x04, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00,\n\t0x52, 0x04, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x46, 0x0a, 0x1e, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e,\n\t0x75, 0x6f, 0x75, 0x73, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f,\n\t0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00,\n\t0x52, 0x1c, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x6f, 0x75, 0x73, 0x43, 0x68, 0x65, 0x63,\n\t0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x1f,\n\t0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x09, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x4d, 0x6f, 0x64, 0x65, 0x12,\n\t0x1d, 0x0a, 0x0a, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0a, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x09, 0x71, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14,\n\t0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x04,\n\t0x70, 0x61, 0x74, 0x68, 0x12, 0x1f, 0x0a, 0x0a, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x09, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x0e, 0x66, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68,\n\t0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x46, 0x75,\n\t0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x66, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x57,\n\t0x72, 0x69, 0x74, 0x65, 0x72, 0x12, 0x4c, 0x0a, 0x0d, 0x66, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68,\n\t0x5f, 0x62, 0x61, 0x74, 0x63, 0x68, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72,\n\t0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x46, 0x75, 0x6e,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x66, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x42, 0x61,\n\t0x74, 0x63, 0x68, 0x12, 0x36, 0x0a, 0x17, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e,\n\t0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x0f,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x15, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67,\n\t0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f,\n\t0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,\n\t0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,\n\t0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67,\n\t0x65, 0x72, 0x42, 0x12, 0x0a, 0x10, 0x73, 0x69, 0x6e, 0x6b, 0x5f, 0x64, 0x65, 0x73, 0x74, 0x69,\n\t0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xb3, 0x01, 0x0a, 0x18, 0x53, 0x74, 0x72, 0x65, 0x61,\n\t0x6d, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x65, 0x61, 0x63, 0x68, 0x46, 0x75, 0x6e, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x0f, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x66, 0x75,\n\t0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74,\n\t0x68, 0x6f, 0x6e, 0x55, 0x44, 0x46, 0x48, 0x00, 0x52, 0x0e, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e,\n\t0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x0e, 0x73, 0x63, 0x61, 0x6c,\n\t0x61, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x55, 0x44, 0x46, 0x48,\n\t0x00, 0x52, 0x0d, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x42, 0x0a, 0x0a, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a,\n\t0x1f, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4f, 0x70, 0x65, 0x72,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x12, 0x42, 0x0a, 0x08, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72,\n\t0x79, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x52, 0x07, 0x71, 0x75, 0x65,\n\t0x72, 0x79, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3c, 0x0a, 0x18, 0x71, 0x75, 0x65, 0x72,\n\t0x79, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f,\n\t0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x15, 0x71, 0x75,\n\t0x65, 0x72, 0x79, 0x53, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4a,\n\t0x73, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x1b, 0x0a, 0x19, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79,\n\t0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6a,\n\t0x73, 0x6f, 0x6e, 0x22, 0x41, 0x0a, 0x18, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67,\n\t0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12,\n\t0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12,\n\t0x15, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x05, 0x72, 0x75, 0x6e, 0x49, 0x64, 0x22, 0xf8, 0x04, 0x0a, 0x15, 0x53, 0x74, 0x72, 0x65, 0x61,\n\t0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x12, 0x42, 0x0a, 0x08, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72,\n\t0x79, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x52, 0x07, 0x71, 0x75, 0x65,\n\t0x72, 0x79, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x25,\n\t0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18,\n\t0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x72, 0x6f,\n\t0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x5f,\n\t0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00,\n\t0x52, 0x0e, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73,\n\t0x12, 0x14, 0x0a, 0x04, 0x73, 0x74, 0x6f, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00,\n\t0x52, 0x04, 0x73, 0x74, 0x6f, 0x70, 0x12, 0x34, 0x0a, 0x15, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73,\n\t0x73, 0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18,\n\t0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73,\n\t0x41, 0x6c, 0x6c, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x4f, 0x0a, 0x07,\n\t0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x2e, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x48, 0x00, 0x52, 0x07, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x12, 0x1e, 0x0a,\n\t0x09, 0x65, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08,\n\t0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x6b, 0x0a,\n\t0x11, 0x61, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69,\n\t0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x41,\n\t0x77, 0x61, 0x69, 0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x10, 0x61, 0x77, 0x61, 0x69, 0x74, 0x54,\n\t0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x2c, 0x0a, 0x0e, 0x45, 0x78,\n\t0x70, 0x6c, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1a, 0x0a, 0x08,\n\t0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08,\n\t0x65, 0x78, 0x74, 0x65, 0x6e, 0x64, 0x65, 0x64, 0x1a, 0x4c, 0x0a, 0x17, 0x41, 0x77, 0x61, 0x69,\n\t0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x12, 0x22, 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d,\n\t0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x6f,\n\t0x75, 0x74, 0x4d, 0x73, 0x88, 0x01, 0x01, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x74, 0x69, 0x6d, 0x65,\n\t0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x22, 0xf5, 0x08, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c,\n\t0x74, 0x12, 0x42, 0x0a, 0x08, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65,\n\t0x72, 0x79, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x52, 0x07, 0x71, 0x75,\n\t0x65, 0x72, 0x79, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c,\n\t0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00,\n\t0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x6a, 0x0a, 0x0f, 0x72, 0x65, 0x63, 0x65,\n\t0x6e, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x3f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x52, 0x65,\n\t0x63, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x67,\n\t0x72, 0x65, 0x73, 0x73, 0x12, 0x54, 0x0a, 0x07, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x18,\n\t0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c,\n\t0x74, 0x2e, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48,\n\t0x00, 0x52, 0x07, 0x65, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x12, 0x5a, 0x0a, 0x09, 0x65, 0x78,\n\t0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x45, 0x78, 0x63, 0x65, 0x70, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x63,\n\t0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x70, 0x0a, 0x11, 0x61, 0x77, 0x61, 0x69, 0x74, 0x5f,\n\t0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x41, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x41, 0x77,\n\t0x61, 0x69, 0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x10, 0x61, 0x77, 0x61, 0x69, 0x74, 0x54, 0x65, 0x72,\n\t0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xaa, 0x01, 0x0a, 0x0c, 0x53, 0x74, 0x61,\n\t0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x73, 0x74, 0x61,\n\t0x74, 0x75, 0x73, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,\n\t0x12, 0x2a, 0x0a, 0x11, 0x69, 0x73, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x61, 0x76, 0x61, 0x69,\n\t0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x44,\n\t0x61, 0x74, 0x61, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2a, 0x0a, 0x11,\n\t0x69, 0x73, 0x5f, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76,\n\t0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x54, 0x72, 0x69, 0x67, 0x67,\n\t0x65, 0x72, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x61,\n\t0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x41,\n\t0x63, 0x74, 0x69, 0x76, 0x65, 0x1a, 0x48, 0x0a, 0x14, 0x52, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x50,\n\t0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x30, 0x0a,\n\t0x14, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73,\n\t0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x72, 0x65, 0x63,\n\t0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4a, 0x73, 0x6f, 0x6e, 0x1a,\n\t0x27, 0x0a, 0x0d, 0x45, 0x78, 0x70, 0x6c, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a, 0xc5, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x63,\n\t0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x30, 0x0a, 0x11,\n\t0x65, 0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x10, 0x65, 0x78, 0x63, 0x65, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x88, 0x01, 0x01, 0x12, 0x24,\n\t0x0a, 0x0b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6c, 0x61, 0x73,\n\t0x73, 0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x72,\n\t0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0a, 0x73, 0x74, 0x61,\n\t0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x88, 0x01, 0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x65,\n\t0x78, 0x63, 0x65, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,\n\t0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73,\n\t0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65,\n\t0x1a, 0x38, 0x0a, 0x16, 0x41, 0x77, 0x61, 0x69, 0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x74, 0x65,\n\t0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a,\n\t0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x42, 0x0d, 0x0a, 0x0b, 0x72, 0x65,\n\t0x73, 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xbd, 0x06, 0x0a, 0x1c, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61,\n\t0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x18, 0x0a, 0x06, 0x61, 0x63,\n\t0x74, 0x69, 0x76, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x06, 0x61, 0x63,\n\t0x74, 0x69, 0x76, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x67, 0x65, 0x74, 0x5f, 0x71, 0x75, 0x65, 0x72,\n\t0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x67, 0x65, 0x74, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x12, 0x7c, 0x0a, 0x15, 0x61, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x61, 0x6e, 0x79,\n\t0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x46, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72,\n\t0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e,\n\t0x41, 0x77, 0x61, 0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48, 0x00, 0x52, 0x13, 0x61, 0x77,\n\t0x61, 0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x12, 0x2b, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69,\n\t0x6e, 0x61, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0f, 0x72,\n\t0x65, 0x73, 0x65, 0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x12, 0x6e,\n\t0x0a, 0x0c, 0x61, 0x64, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x18, 0x05,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79,\n\t0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x48,\n\t0x00, 0x52, 0x0b, 0x61, 0x64, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x12, 0x74,\n\t0x0a, 0x0f, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65,\n\t0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e,\n\t0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4c, 0x69, 0x73, 0x74,\n\t0x65, 0x6e, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0e, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x6c, 0x69, 0x73,\n\t0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0d,\n\t0x6c, 0x69, 0x73, 0x74, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x1a, 0x4f, 0x0a,\n\t0x1a, 0x41, 0x77, 0x61, 0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x22, 0x0a, 0x0a, 0x74,\n\t0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x48,\n\t0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x88, 0x01, 0x01, 0x42,\n\t0x0d, 0x0a, 0x0b, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x6d, 0x73, 0x1a, 0xcd,\n\t0x01, 0x0a, 0x1d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72,\n\t0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x12, 0x29, 0x0a, 0x10, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x79,\n\t0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74,\n\t0x65, 0x6e, 0x65, 0x72, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x55, 0x0a, 0x17, 0x70,\n\t0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x70,\n\t0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74,\n\t0x68, 0x6f, 0x6e, 0x55, 0x44, 0x46, 0x48, 0x00, 0x52, 0x15, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e,\n\t0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x88,\n\t0x01, 0x01, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02,\n\t0x69, 0x64, 0x42, 0x1a, 0x0a, 0x18, 0x5f, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x6c, 0x69,\n\t0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x09,\n\t0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x22, 0xb4, 0x08, 0x0a, 0x22, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61,\n\t0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x12, 0x58, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x3e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d,\n\t0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73,\n\t0x75, 0x6c, 0x74, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74,\n\t0x48, 0x00, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x12, 0x60, 0x0a, 0x05, 0x71, 0x75,\n\t0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x48, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,\n\t0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x53, 0x74, 0x72,\n\t0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x73, 0x74, 0x61,\n\t0x6e, 0x63, 0x65, 0x48, 0x00, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12, 0x81, 0x01, 0x0a,\n\t0x15, 0x61, 0x77, 0x61, 0x69, 0x74, 0x5f, 0x61, 0x6e, 0x79, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69,\n\t0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72,\n\t0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67,\n\t0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e,\n\t0x41, 0x77, 0x61, 0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x13, 0x61, 0x77, 0x61,\n\t0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x12, 0x2b, 0x0a, 0x10, 0x72, 0x65, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e,\n\t0x61, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0f, 0x72, 0x65,\n\t0x73, 0x65, 0x74, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x12, 0x23, 0x0a,\n\t0x0c, 0x61, 0x64, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x18, 0x05, 0x20,\n\t0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x64, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x12, 0x29, 0x0a, 0x0f, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x6c, 0x69, 0x73,\n\t0x74, 0x65, 0x6e, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0e, 0x72,\n\t0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x12, 0x7b, 0x0a,\n\t0x0e, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18,\n\t0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x52, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x72,\n\t0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65,\n\t0x6e, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x6c, 0x69, 0x73,\n\t0x74, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x1a, 0x7f, 0x0a, 0x0c, 0x41, 0x63,\n\t0x74, 0x69, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x6f, 0x0a, 0x0e, 0x61, 0x63,\n\t0x74, 0x69, 0x76, 0x65, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03,\n\t0x28, 0x0b, 0x32, 0x48, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72,\n\t0x79, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52,\n\t0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51,\n\t0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x0d, 0x61, 0x63,\n\t0x74, 0x69, 0x76, 0x65, 0x51, 0x75, 0x65, 0x72, 0x69, 0x65, 0x73, 0x1a, 0x73, 0x0a, 0x16, 0x53,\n\t0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x73,\n\t0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x37, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79,\n\t0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x52, 0x02, 0x69, 0x64, 0x12, 0x17,\n\t0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x04,\n\t0x6e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x1a, 0x3b, 0x0a, 0x19, 0x41, 0x77, 0x61, 0x69, 0x74, 0x41, 0x6e, 0x79, 0x54, 0x65, 0x72, 0x6d,\n\t0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1e, 0x0a,\n\t0x0a, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x08, 0x52, 0x0a, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x1a, 0x4b, 0x0a,\n\t0x1e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c,\n\t0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12,\n\t0x29, 0x0a, 0x10, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x79, 0x6c,\n\t0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0f, 0x6c, 0x69, 0x73, 0x74, 0x65,\n\t0x6e, 0x65, 0x72, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x1a, 0x45, 0x0a, 0x20, 0x4c, 0x69,\n\t0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79,\n\t0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x21,\n\t0x0a, 0x0c, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x49, 0x64,\n\t0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65,\n\t0x22, 0xad, 0x01, 0x0a, 0x20, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x42, 0x75, 0x73, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x3b, 0x0a, 0x19, 0x61, 0x64, 0x64, 0x5f, 0x6c, 0x69, 0x73,\n\t0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x16, 0x61, 0x64, 0x64, 0x4c,\n\t0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x42, 0x75, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x12, 0x41, 0x0a, 0x1c, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x6c, 0x69, 0x73,\n\t0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x19, 0x72, 0x65, 0x6d, 0x6f,\n\t0x76, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x42, 0x75, 0x73, 0x4c, 0x69, 0x73,\n\t0x74, 0x65, 0x6e, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x22, 0x83, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75,\n\t0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74,\n\t0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4a, 0x73, 0x6f, 0x6e, 0x12,\n\t0x45, 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65,\n\t0x72, 0x79, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65,\n\t0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0xcc, 0x01, 0x0a, 0x22, 0x53, 0x74, 0x72, 0x65, 0x61,\n\t0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65,\n\t0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x42, 0x0a,\n\t0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74,\n\t0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4c, 0x69, 0x73, 0x74,\n\t0x65, 0x6e, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74,\n\t0x73, 0x12, 0x42, 0x0a, 0x1b, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x62, 0x75,\n\t0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x65, 0x64,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x18, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x42, 0x75, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x41, 0x64, 0x64,\n\t0x65, 0x64, 0x88, 0x01, 0x01, 0x42, 0x1e, 0x0a, 0x1c, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e,\n\t0x65, 0x72, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x5f,\n\t0x61, 0x64, 0x64, 0x65, 0x64, 0x22, 0x15, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x22, 0xd4, 0x01, 0x0a,\n\t0x19, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x55, 0x0a, 0x09, 0x72, 0x65,\n\t0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x47, 0x65,\n\t0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,\n\t0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,\n\t0x73, 0x1a, 0x60, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x45, 0x6e,\n\t0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66,\n\t0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,\n\t0x02, 0x38, 0x01, 0x22, 0x58, 0x0a, 0x1c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73,\n\t0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x12, 0x38, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f,\n\t0x66, 0x69, 0x6c, 0x65, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x22, 0x43, 0x0a,\n\t0x22, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50,\n\t0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73,\n\t0x75, 0x6c, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69,\n\t0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c, 0x65,\n\t0x49, 0x64, 0x22, 0x64, 0x0a, 0x21, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x43, 0x61, 0x63, 0x68,\n\t0x65, 0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x3f, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x64,\n\t0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08,\n\t0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xcd, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x65,\n\t0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x33,\n\t0x0a, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x72, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x08, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x61, 0x67,\n\t0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x65, 0x61, 0x67, 0x65, 0x72, 0x12,\n\t0x45, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c,\n\t0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65,\n\t0x76, 0x65, 0x6c, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65,\n\t0x76, 0x65, 0x6c, 0x88, 0x01, 0x01, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x73, 0x74, 0x6f, 0x72, 0x61,\n\t0x67, 0x65, 0x5f, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x22, 0xe8, 0x03, 0x0a, 0x15, 0x4d, 0x65, 0x72,\n\t0x67, 0x65, 0x49, 0x6e, 0x74, 0x6f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x12, 0x2a, 0x0a, 0x11, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x74, 0x61, 0x62,\n\t0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x74,\n\t0x61, 0x72, 0x67, 0x65, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x43,\n\t0x0a, 0x11, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70,\n\t0x6c, 0x61, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50,\n\t0x6c, 0x61, 0x6e, 0x12, 0x42, 0x0a, 0x0f, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6e,\n\t0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x43, 0x6f,\n\t0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x0d, 0x6d, 0x61, 0x74, 0x63, 0x68,\n\t0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x6d, 0x61, 0x74, 0x63, 0x68,\n\t0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x49, 0x0a, 0x13, 0x6e, 0x6f, 0x74, 0x5f, 0x6d,\n\t0x61, 0x74, 0x63, 0x68, 0x65, 0x64, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52,\n\t0x11, 0x6e, 0x6f, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x73, 0x12, 0x5b, 0x0a, 0x1d, 0x6e, 0x6f, 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65,\n\t0x64, 0x5f, 0x62, 0x79, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x19, 0x6e, 0x6f, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x64,\n\t0x42, 0x79, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12,\n\t0x32, 0x0a, 0x15, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x5f, 0x65,\n\t0x76, 0x6f, 0x6c, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x13,\n\t0x77, 0x69, 0x74, 0x68, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x45, 0x76, 0x6f, 0x6c, 0x75, 0x74,\n\t0x69, 0x6f, 0x6e, 0x22, 0xd4, 0x01, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x45,\n\t0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x16,\n\t0x0a, 0x06, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06,\n\t0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e,\n\t0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x12, 0x4c, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x32, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61,\n\t0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x3a,\n\t0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10,\n\t0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79,\n\t0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x2a, 0x85, 0x01, 0x0a, 0x17, 0x53,\n\t0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x45, 0x76, 0x65,\n\t0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x1a, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f,\n\t0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49,\n\t0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f,\n\t0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, 0x01,\n\t0x12, 0x1a, 0x0a, 0x16, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e,\n\t0x41, 0x54, 0x45, 0x44, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10,\n\t0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x49, 0x44, 0x4c, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,\n\t0x10, 0x03, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70,\n\t0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,\n\t0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,\n\t0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_commands_proto_rawDescOnce sync.Once\n\tfile_spark_connect_commands_proto_rawDescData = file_spark_connect_commands_proto_rawDesc\n)\n\nfunc file_spark_connect_commands_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_commands_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_commands_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_commands_proto_rawDescData)\n\t})\n\treturn file_spark_connect_commands_proto_rawDescData\n}\n\nvar file_spark_connect_commands_proto_enumTypes = make([]protoimpl.EnumInfo, 4)\nvar file_spark_connect_commands_proto_msgTypes = make([]protoimpl.MessageInfo, 48)\nvar file_spark_connect_commands_proto_goTypes = []interface{}{\n\t(StreamingQueryEventType)(0),                               // 0: spark.connect.StreamingQueryEventType\n\t(WriteOperation_SaveMode)(0),                               // 1: spark.connect.WriteOperation.SaveMode\n\t(WriteOperation_SaveTable_TableSaveMethod)(0),              // 2: spark.connect.WriteOperation.SaveTable.TableSaveMethod\n\t(WriteOperationV2_Mode)(0),                                 // 3: spark.connect.WriteOperationV2.Mode\n\t(*Command)(nil),                                            // 4: spark.connect.Command\n\t(*SqlCommand)(nil),                                         // 5: spark.connect.SqlCommand\n\t(*CreateDataFrameViewCommand)(nil),                         // 6: spark.connect.CreateDataFrameViewCommand\n\t(*WriteOperation)(nil),                                     // 7: spark.connect.WriteOperation\n\t(*WriteOperationV2)(nil),                                   // 8: spark.connect.WriteOperationV2\n\t(*WriteStreamOperationStart)(nil),                          // 9: spark.connect.WriteStreamOperationStart\n\t(*StreamingForeachFunction)(nil),                           // 10: spark.connect.StreamingForeachFunction\n\t(*WriteStreamOperationStartResult)(nil),                    // 11: spark.connect.WriteStreamOperationStartResult\n\t(*StreamingQueryInstanceId)(nil),                           // 12: spark.connect.StreamingQueryInstanceId\n\t(*StreamingQueryCommand)(nil),                              // 13: spark.connect.StreamingQueryCommand\n\t(*StreamingQueryCommandResult)(nil),                        // 14: spark.connect.StreamingQueryCommandResult\n\t(*StreamingQueryManagerCommand)(nil),                       // 15: spark.connect.StreamingQueryManagerCommand\n\t(*StreamingQueryManagerCommandResult)(nil),                 // 16: spark.connect.StreamingQueryManagerCommandResult\n\t(*StreamingQueryListenerBusCommand)(nil),                   // 17: spark.connect.StreamingQueryListenerBusCommand\n\t(*StreamingQueryListenerEvent)(nil),                        // 18: spark.connect.StreamingQueryListenerEvent\n\t(*StreamingQueryListenerEventsResult)(nil),                 // 19: spark.connect.StreamingQueryListenerEventsResult\n\t(*GetResourcesCommand)(nil),                                // 20: spark.connect.GetResourcesCommand\n\t(*GetResourcesCommandResult)(nil),                          // 21: spark.connect.GetResourcesCommandResult\n\t(*CreateResourceProfileCommand)(nil),                       // 22: spark.connect.CreateResourceProfileCommand\n\t(*CreateResourceProfileCommandResult)(nil),                 // 23: spark.connect.CreateResourceProfileCommandResult\n\t(*RemoveCachedRemoteRelationCommand)(nil),                  // 24: spark.connect.RemoveCachedRemoteRelationCommand\n\t(*CheckpointCommand)(nil),                                  // 25: spark.connect.CheckpointCommand\n\t(*MergeIntoTableCommand)(nil),                              // 26: spark.connect.MergeIntoTableCommand\n\t(*ExecuteExternalCommand)(nil),                             // 27: spark.connect.ExecuteExternalCommand\n\tnil,                                                        // 28: spark.connect.SqlCommand.ArgsEntry\n\tnil,                                                        // 29: spark.connect.SqlCommand.NamedArgumentsEntry\n\tnil,                                                        // 30: spark.connect.WriteOperation.OptionsEntry\n\t(*WriteOperation_SaveTable)(nil),                           // 31: spark.connect.WriteOperation.SaveTable\n\t(*WriteOperation_BucketBy)(nil),                            // 32: spark.connect.WriteOperation.BucketBy\n\tnil,                                                        // 33: spark.connect.WriteOperationV2.OptionsEntry\n\tnil,                                                        // 34: spark.connect.WriteOperationV2.TablePropertiesEntry\n\tnil,                                                        // 35: spark.connect.WriteStreamOperationStart.OptionsEntry\n\t(*StreamingQueryCommand_ExplainCommand)(nil),               // 36: spark.connect.StreamingQueryCommand.ExplainCommand\n\t(*StreamingQueryCommand_AwaitTerminationCommand)(nil),      // 37: spark.connect.StreamingQueryCommand.AwaitTerminationCommand\n\t(*StreamingQueryCommandResult_StatusResult)(nil),           // 38: spark.connect.StreamingQueryCommandResult.StatusResult\n\t(*StreamingQueryCommandResult_RecentProgressResult)(nil),   // 39: spark.connect.StreamingQueryCommandResult.RecentProgressResult\n\t(*StreamingQueryCommandResult_ExplainResult)(nil),          // 40: spark.connect.StreamingQueryCommandResult.ExplainResult\n\t(*StreamingQueryCommandResult_ExceptionResult)(nil),        // 41: spark.connect.StreamingQueryCommandResult.ExceptionResult\n\t(*StreamingQueryCommandResult_AwaitTerminationResult)(nil), // 42: spark.connect.StreamingQueryCommandResult.AwaitTerminationResult\n\t(*StreamingQueryManagerCommand_AwaitAnyTerminationCommand)(nil),             // 43: spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommand\n\t(*StreamingQueryManagerCommand_StreamingQueryListenerCommand)(nil),          // 44: spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommand\n\t(*StreamingQueryManagerCommandResult_ActiveResult)(nil),                     // 45: spark.connect.StreamingQueryManagerCommandResult.ActiveResult\n\t(*StreamingQueryManagerCommandResult_StreamingQueryInstance)(nil),           // 46: spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstance\n\t(*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult)(nil),        // 47: spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResult\n\t(*StreamingQueryManagerCommandResult_StreamingQueryListenerInstance)(nil),   // 48: spark.connect.StreamingQueryManagerCommandResult.StreamingQueryListenerInstance\n\t(*StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult)(nil), // 49: spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResult\n\tnil,                                     // 50: spark.connect.GetResourcesCommandResult.ResourcesEntry\n\tnil,                                     // 51: spark.connect.ExecuteExternalCommand.OptionsEntry\n\t(*CommonInlineUserDefinedFunction)(nil), // 52: spark.connect.CommonInlineUserDefinedFunction\n\t(*CommonInlineUserDefinedTableFunction)(nil), // 53: spark.connect.CommonInlineUserDefinedTableFunction\n\t(*CommonInlineUserDefinedDataSource)(nil),    // 54: spark.connect.CommonInlineUserDefinedDataSource\n\t(*MlCommand)(nil),                            // 55: spark.connect.MlCommand\n\t(*PipelineCommand)(nil),                      // 56: spark.connect.PipelineCommand\n\t(*anypb.Any)(nil),                            // 57: google.protobuf.Any\n\t(*Expression_Literal)(nil),                   // 58: spark.connect.Expression.Literal\n\t(*Expression)(nil),                           // 59: spark.connect.Expression\n\t(*Relation)(nil),                             // 60: spark.connect.Relation\n\t(*PythonUDF)(nil),                            // 61: spark.connect.PythonUDF\n\t(*ScalarScalaUDF)(nil),                       // 62: spark.connect.ScalarScalaUDF\n\t(*ResourceProfile)(nil),                      // 63: spark.connect.ResourceProfile\n\t(*CachedRemoteRelation)(nil),                 // 64: spark.connect.CachedRemoteRelation\n\t(*StorageLevel)(nil),                         // 65: spark.connect.StorageLevel\n\t(*ResourceInformation)(nil),                  // 66: spark.connect.ResourceInformation\n}\nvar file_spark_connect_commands_proto_depIdxs = []int32{\n\t52, // 0: spark.connect.Command.register_function:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t7,  // 1: spark.connect.Command.write_operation:type_name -> spark.connect.WriteOperation\n\t6,  // 2: spark.connect.Command.create_dataframe_view:type_name -> spark.connect.CreateDataFrameViewCommand\n\t8,  // 3: spark.connect.Command.write_operation_v2:type_name -> spark.connect.WriteOperationV2\n\t5,  // 4: spark.connect.Command.sql_command:type_name -> spark.connect.SqlCommand\n\t9,  // 5: spark.connect.Command.write_stream_operation_start:type_name -> spark.connect.WriteStreamOperationStart\n\t13, // 6: spark.connect.Command.streaming_query_command:type_name -> spark.connect.StreamingQueryCommand\n\t20, // 7: spark.connect.Command.get_resources_command:type_name -> spark.connect.GetResourcesCommand\n\t15, // 8: spark.connect.Command.streaming_query_manager_command:type_name -> spark.connect.StreamingQueryManagerCommand\n\t53, // 9: spark.connect.Command.register_table_function:type_name -> spark.connect.CommonInlineUserDefinedTableFunction\n\t17, // 10: spark.connect.Command.streaming_query_listener_bus_command:type_name -> spark.connect.StreamingQueryListenerBusCommand\n\t54, // 11: spark.connect.Command.register_data_source:type_name -> spark.connect.CommonInlineUserDefinedDataSource\n\t22, // 12: spark.connect.Command.create_resource_profile_command:type_name -> spark.connect.CreateResourceProfileCommand\n\t25, // 13: spark.connect.Command.checkpoint_command:type_name -> spark.connect.CheckpointCommand\n\t24, // 14: spark.connect.Command.remove_cached_remote_relation_command:type_name -> spark.connect.RemoveCachedRemoteRelationCommand\n\t26, // 15: spark.connect.Command.merge_into_table_command:type_name -> spark.connect.MergeIntoTableCommand\n\t55, // 16: spark.connect.Command.ml_command:type_name -> spark.connect.MlCommand\n\t27, // 17: spark.connect.Command.execute_external_command:type_name -> spark.connect.ExecuteExternalCommand\n\t56, // 18: spark.connect.Command.pipeline_command:type_name -> spark.connect.PipelineCommand\n\t57, // 19: spark.connect.Command.extension:type_name -> google.protobuf.Any\n\t28, // 20: spark.connect.SqlCommand.args:type_name -> spark.connect.SqlCommand.ArgsEntry\n\t58, // 21: spark.connect.SqlCommand.pos_args:type_name -> spark.connect.Expression.Literal\n\t29, // 22: spark.connect.SqlCommand.named_arguments:type_name -> spark.connect.SqlCommand.NamedArgumentsEntry\n\t59, // 23: spark.connect.SqlCommand.pos_arguments:type_name -> spark.connect.Expression\n\t60, // 24: spark.connect.SqlCommand.input:type_name -> spark.connect.Relation\n\t60, // 25: spark.connect.CreateDataFrameViewCommand.input:type_name -> spark.connect.Relation\n\t60, // 26: spark.connect.WriteOperation.input:type_name -> spark.connect.Relation\n\t31, // 27: spark.connect.WriteOperation.table:type_name -> spark.connect.WriteOperation.SaveTable\n\t1,  // 28: spark.connect.WriteOperation.mode:type_name -> spark.connect.WriteOperation.SaveMode\n\t32, // 29: spark.connect.WriteOperation.bucket_by:type_name -> spark.connect.WriteOperation.BucketBy\n\t30, // 30: spark.connect.WriteOperation.options:type_name -> spark.connect.WriteOperation.OptionsEntry\n\t60, // 31: spark.connect.WriteOperationV2.input:type_name -> spark.connect.Relation\n\t59, // 32: spark.connect.WriteOperationV2.partitioning_columns:type_name -> spark.connect.Expression\n\t33, // 33: spark.connect.WriteOperationV2.options:type_name -> spark.connect.WriteOperationV2.OptionsEntry\n\t34, // 34: spark.connect.WriteOperationV2.table_properties:type_name -> spark.connect.WriteOperationV2.TablePropertiesEntry\n\t3,  // 35: spark.connect.WriteOperationV2.mode:type_name -> spark.connect.WriteOperationV2.Mode\n\t59, // 36: spark.connect.WriteOperationV2.overwrite_condition:type_name -> spark.connect.Expression\n\t60, // 37: spark.connect.WriteStreamOperationStart.input:type_name -> spark.connect.Relation\n\t35, // 38: spark.connect.WriteStreamOperationStart.options:type_name -> spark.connect.WriteStreamOperationStart.OptionsEntry\n\t10, // 39: spark.connect.WriteStreamOperationStart.foreach_writer:type_name -> spark.connect.StreamingForeachFunction\n\t10, // 40: spark.connect.WriteStreamOperationStart.foreach_batch:type_name -> spark.connect.StreamingForeachFunction\n\t61, // 41: spark.connect.StreamingForeachFunction.python_function:type_name -> spark.connect.PythonUDF\n\t62, // 42: spark.connect.StreamingForeachFunction.scala_function:type_name -> spark.connect.ScalarScalaUDF\n\t12, // 43: spark.connect.WriteStreamOperationStartResult.query_id:type_name -> spark.connect.StreamingQueryInstanceId\n\t12, // 44: spark.connect.StreamingQueryCommand.query_id:type_name -> spark.connect.StreamingQueryInstanceId\n\t36, // 45: spark.connect.StreamingQueryCommand.explain:type_name -> spark.connect.StreamingQueryCommand.ExplainCommand\n\t37, // 46: spark.connect.StreamingQueryCommand.await_termination:type_name -> spark.connect.StreamingQueryCommand.AwaitTerminationCommand\n\t12, // 47: spark.connect.StreamingQueryCommandResult.query_id:type_name -> spark.connect.StreamingQueryInstanceId\n\t38, // 48: spark.connect.StreamingQueryCommandResult.status:type_name -> spark.connect.StreamingQueryCommandResult.StatusResult\n\t39, // 49: spark.connect.StreamingQueryCommandResult.recent_progress:type_name -> spark.connect.StreamingQueryCommandResult.RecentProgressResult\n\t40, // 50: spark.connect.StreamingQueryCommandResult.explain:type_name -> spark.connect.StreamingQueryCommandResult.ExplainResult\n\t41, // 51: spark.connect.StreamingQueryCommandResult.exception:type_name -> spark.connect.StreamingQueryCommandResult.ExceptionResult\n\t42, // 52: spark.connect.StreamingQueryCommandResult.await_termination:type_name -> spark.connect.StreamingQueryCommandResult.AwaitTerminationResult\n\t43, // 53: spark.connect.StreamingQueryManagerCommand.await_any_termination:type_name -> spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommand\n\t44, // 54: spark.connect.StreamingQueryManagerCommand.add_listener:type_name -> spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommand\n\t44, // 55: spark.connect.StreamingQueryManagerCommand.remove_listener:type_name -> spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommand\n\t45, // 56: spark.connect.StreamingQueryManagerCommandResult.active:type_name -> spark.connect.StreamingQueryManagerCommandResult.ActiveResult\n\t46, // 57: spark.connect.StreamingQueryManagerCommandResult.query:type_name -> spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstance\n\t47, // 58: spark.connect.StreamingQueryManagerCommandResult.await_any_termination:type_name -> spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResult\n\t49, // 59: spark.connect.StreamingQueryManagerCommandResult.list_listeners:type_name -> spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResult\n\t0,  // 60: spark.connect.StreamingQueryListenerEvent.event_type:type_name -> spark.connect.StreamingQueryEventType\n\t18, // 61: spark.connect.StreamingQueryListenerEventsResult.events:type_name -> spark.connect.StreamingQueryListenerEvent\n\t50, // 62: spark.connect.GetResourcesCommandResult.resources:type_name -> spark.connect.GetResourcesCommandResult.ResourcesEntry\n\t63, // 63: spark.connect.CreateResourceProfileCommand.profile:type_name -> spark.connect.ResourceProfile\n\t64, // 64: spark.connect.RemoveCachedRemoteRelationCommand.relation:type_name -> spark.connect.CachedRemoteRelation\n\t60, // 65: spark.connect.CheckpointCommand.relation:type_name -> spark.connect.Relation\n\t65, // 66: spark.connect.CheckpointCommand.storage_level:type_name -> spark.connect.StorageLevel\n\t60, // 67: spark.connect.MergeIntoTableCommand.source_table_plan:type_name -> spark.connect.Relation\n\t59, // 68: spark.connect.MergeIntoTableCommand.merge_condition:type_name -> spark.connect.Expression\n\t59, // 69: spark.connect.MergeIntoTableCommand.match_actions:type_name -> spark.connect.Expression\n\t59, // 70: spark.connect.MergeIntoTableCommand.not_matched_actions:type_name -> spark.connect.Expression\n\t59, // 71: spark.connect.MergeIntoTableCommand.not_matched_by_source_actions:type_name -> spark.connect.Expression\n\t51, // 72: spark.connect.ExecuteExternalCommand.options:type_name -> spark.connect.ExecuteExternalCommand.OptionsEntry\n\t58, // 73: spark.connect.SqlCommand.ArgsEntry.value:type_name -> spark.connect.Expression.Literal\n\t59, // 74: spark.connect.SqlCommand.NamedArgumentsEntry.value:type_name -> spark.connect.Expression\n\t2,  // 75: spark.connect.WriteOperation.SaveTable.save_method:type_name -> spark.connect.WriteOperation.SaveTable.TableSaveMethod\n\t61, // 76: spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommand.python_listener_payload:type_name -> spark.connect.PythonUDF\n\t46, // 77: spark.connect.StreamingQueryManagerCommandResult.ActiveResult.active_queries:type_name -> spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstance\n\t12, // 78: spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstance.id:type_name -> spark.connect.StreamingQueryInstanceId\n\t66, // 79: spark.connect.GetResourcesCommandResult.ResourcesEntry.value:type_name -> spark.connect.ResourceInformation\n\t80, // [80:80] is the sub-list for method output_type\n\t80, // [80:80] is the sub-list for method input_type\n\t80, // [80:80] is the sub-list for extension type_name\n\t80, // [80:80] is the sub-list for extension extendee\n\t0,  // [0:80] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_commands_proto_init() }\nfunc file_spark_connect_commands_proto_init() {\n\tif File_spark_connect_commands_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_common_proto_init()\n\tfile_spark_connect_expressions_proto_init()\n\tfile_spark_connect_relations_proto_init()\n\tfile_spark_connect_ml_proto_init()\n\tfile_spark_connect_pipelines_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_commands_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Command); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SqlCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CreateDataFrameViewCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteOperation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteOperationV2); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteStreamOperationStart); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingForeachFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteStreamOperationStartResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryInstanceId); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryListenerBusCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryListenerEvent); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryListenerEventsResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GetResourcesCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GetResourcesCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CreateResourceProfileCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CreateResourceProfileCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RemoveCachedRemoteRelationCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CheckpointCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MergeIntoTableCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecuteExternalCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteOperation_SaveTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WriteOperation_BucketBy); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommand_ExplainCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommand_AwaitTerminationCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult_StatusResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult_RecentProgressResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult_ExplainResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult_ExceptionResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryCommandResult_AwaitTerminationResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommand_AwaitAnyTerminationCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommand_StreamingQueryListenerCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult_ActiveResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult_StreamingQueryInstance); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult_StreamingQueryListenerInstance); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_commands_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*Command_RegisterFunction)(nil),\n\t\t(*Command_WriteOperation)(nil),\n\t\t(*Command_CreateDataframeView)(nil),\n\t\t(*Command_WriteOperationV2)(nil),\n\t\t(*Command_SqlCommand)(nil),\n\t\t(*Command_WriteStreamOperationStart)(nil),\n\t\t(*Command_StreamingQueryCommand)(nil),\n\t\t(*Command_GetResourcesCommand)(nil),\n\t\t(*Command_StreamingQueryManagerCommand)(nil),\n\t\t(*Command_RegisterTableFunction)(nil),\n\t\t(*Command_StreamingQueryListenerBusCommand)(nil),\n\t\t(*Command_RegisterDataSource)(nil),\n\t\t(*Command_CreateResourceProfileCommand)(nil),\n\t\t(*Command_CheckpointCommand)(nil),\n\t\t(*Command_RemoveCachedRemoteRelationCommand)(nil),\n\t\t(*Command_MergeIntoTableCommand)(nil),\n\t\t(*Command_MlCommand)(nil),\n\t\t(*Command_ExecuteExternalCommand)(nil),\n\t\t(*Command_PipelineCommand)(nil),\n\t\t(*Command_Extension)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[3].OneofWrappers = []interface{}{\n\t\t(*WriteOperation_Path)(nil),\n\t\t(*WriteOperation_Table)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[4].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[5].OneofWrappers = []interface{}{\n\t\t(*WriteStreamOperationStart_ProcessingTimeInterval)(nil),\n\t\t(*WriteStreamOperationStart_AvailableNow)(nil),\n\t\t(*WriteStreamOperationStart_Once)(nil),\n\t\t(*WriteStreamOperationStart_ContinuousCheckpointInterval)(nil),\n\t\t(*WriteStreamOperationStart_Path)(nil),\n\t\t(*WriteStreamOperationStart_TableName)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[6].OneofWrappers = []interface{}{\n\t\t(*StreamingForeachFunction_PythonFunction)(nil),\n\t\t(*StreamingForeachFunction_ScalaFunction)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[7].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[9].OneofWrappers = []interface{}{\n\t\t(*StreamingQueryCommand_Status)(nil),\n\t\t(*StreamingQueryCommand_LastProgress)(nil),\n\t\t(*StreamingQueryCommand_RecentProgress)(nil),\n\t\t(*StreamingQueryCommand_Stop)(nil),\n\t\t(*StreamingQueryCommand_ProcessAllAvailable)(nil),\n\t\t(*StreamingQueryCommand_Explain)(nil),\n\t\t(*StreamingQueryCommand_Exception)(nil),\n\t\t(*StreamingQueryCommand_AwaitTermination)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[10].OneofWrappers = []interface{}{\n\t\t(*StreamingQueryCommandResult_Status)(nil),\n\t\t(*StreamingQueryCommandResult_RecentProgress)(nil),\n\t\t(*StreamingQueryCommandResult_Explain)(nil),\n\t\t(*StreamingQueryCommandResult_Exception)(nil),\n\t\t(*StreamingQueryCommandResult_AwaitTermination)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[11].OneofWrappers = []interface{}{\n\t\t(*StreamingQueryManagerCommand_Active)(nil),\n\t\t(*StreamingQueryManagerCommand_GetQuery)(nil),\n\t\t(*StreamingQueryManagerCommand_AwaitAnyTermination)(nil),\n\t\t(*StreamingQueryManagerCommand_ResetTerminated)(nil),\n\t\t(*StreamingQueryManagerCommand_AddListener)(nil),\n\t\t(*StreamingQueryManagerCommand_RemoveListener)(nil),\n\t\t(*StreamingQueryManagerCommand_ListListeners)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[12].OneofWrappers = []interface{}{\n\t\t(*StreamingQueryManagerCommandResult_Active)(nil),\n\t\t(*StreamingQueryManagerCommandResult_Query)(nil),\n\t\t(*StreamingQueryManagerCommandResult_AwaitAnyTermination)(nil),\n\t\t(*StreamingQueryManagerCommandResult_ResetTerminated)(nil),\n\t\t(*StreamingQueryManagerCommandResult_AddListener)(nil),\n\t\t(*StreamingQueryManagerCommandResult_RemoveListener)(nil),\n\t\t(*StreamingQueryManagerCommandResult_ListListeners)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[13].OneofWrappers = []interface{}{\n\t\t(*StreamingQueryListenerBusCommand_AddListenerBusListener)(nil),\n\t\t(*StreamingQueryListenerBusCommand_RemoveListenerBusListener)(nil),\n\t}\n\tfile_spark_connect_commands_proto_msgTypes[15].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[21].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[33].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[37].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[39].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[40].OneofWrappers = []interface{}{}\n\tfile_spark_connect_commands_proto_msgTypes[42].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_commands_proto_rawDesc,\n\t\t\tNumEnums:      4,\n\t\t\tNumMessages:   48,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_commands_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_commands_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_commands_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_commands_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_commands_proto = out.File\n\tfile_spark_connect_commands_proto_rawDesc = nil\n\tfile_spark_connect_commands_proto_goTypes = nil\n\tfile_spark_connect_commands_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/common.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/common.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// StorageLevel for persisting Datasets/Tables.\ntype StorageLevel struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Whether the cache should use disk or not.\n\tUseDisk bool `protobuf:\"varint,1,opt,name=use_disk,json=useDisk,proto3\" json:\"use_disk,omitempty\"`\n\t// (Required) Whether the cache should use memory or not.\n\tUseMemory bool `protobuf:\"varint,2,opt,name=use_memory,json=useMemory,proto3\" json:\"use_memory,omitempty\"`\n\t// (Required) Whether the cache should use off-heap or not.\n\tUseOffHeap bool `protobuf:\"varint,3,opt,name=use_off_heap,json=useOffHeap,proto3\" json:\"use_off_heap,omitempty\"`\n\t// (Required) Whether the cached data is deserialized or not.\n\tDeserialized bool `protobuf:\"varint,4,opt,name=deserialized,proto3\" json:\"deserialized,omitempty\"`\n\t// (Required) The number of replicas.\n\tReplication int32 `protobuf:\"varint,5,opt,name=replication,proto3\" json:\"replication,omitempty\"`\n}\n\nfunc (x *StorageLevel) Reset() {\n\t*x = StorageLevel{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StorageLevel) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StorageLevel) ProtoMessage() {}\n\nfunc (x *StorageLevel) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StorageLevel.ProtoReflect.Descriptor instead.\nfunc (*StorageLevel) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (x *StorageLevel) GetUseDisk() bool {\n\tif x != nil {\n\t\treturn x.UseDisk\n\t}\n\treturn false\n}\n\nfunc (x *StorageLevel) GetUseMemory() bool {\n\tif x != nil {\n\t\treturn x.UseMemory\n\t}\n\treturn false\n}\n\nfunc (x *StorageLevel) GetUseOffHeap() bool {\n\tif x != nil {\n\t\treturn x.UseOffHeap\n\t}\n\treturn false\n}\n\nfunc (x *StorageLevel) GetDeserialized() bool {\n\tif x != nil {\n\t\treturn x.Deserialized\n\t}\n\treturn false\n}\n\nfunc (x *StorageLevel) GetReplication() int32 {\n\tif x != nil {\n\t\treturn x.Replication\n\t}\n\treturn 0\n}\n\n// ResourceInformation to hold information about a type of Resource.\n// The corresponding class is 'org.apache.spark.resource.ResourceInformation'\ntype ResourceInformation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The name of the resource\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Required) An array of strings describing the addresses of the resource.\n\tAddresses []string `protobuf:\"bytes,2,rep,name=addresses,proto3\" json:\"addresses,omitempty\"`\n}\n\nfunc (x *ResourceInformation) Reset() {\n\t*x = ResourceInformation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ResourceInformation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ResourceInformation) ProtoMessage() {}\n\nfunc (x *ResourceInformation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ResourceInformation.ProtoReflect.Descriptor instead.\nfunc (*ResourceInformation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *ResourceInformation) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *ResourceInformation) GetAddresses() []string {\n\tif x != nil {\n\t\treturn x.Addresses\n\t}\n\treturn nil\n}\n\n// An executor resource request.\ntype ExecutorResourceRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) resource name.\n\tResourceName string `protobuf:\"bytes,1,opt,name=resource_name,json=resourceName,proto3\" json:\"resource_name,omitempty\"`\n\t// (Required) resource amount requesting.\n\tAmount int64 `protobuf:\"varint,2,opt,name=amount,proto3\" json:\"amount,omitempty\"`\n\t// Optional script used to discover the resources.\n\tDiscoveryScript *string `protobuf:\"bytes,3,opt,name=discovery_script,json=discoveryScript,proto3,oneof\" json:\"discovery_script,omitempty\"`\n\t// Optional vendor, required for some cluster managers.\n\tVendor *string `protobuf:\"bytes,4,opt,name=vendor,proto3,oneof\" json:\"vendor,omitempty\"`\n}\n\nfunc (x *ExecutorResourceRequest) Reset() {\n\t*x = ExecutorResourceRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExecutorResourceRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExecutorResourceRequest) ProtoMessage() {}\n\nfunc (x *ExecutorResourceRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExecutorResourceRequest.ProtoReflect.Descriptor instead.\nfunc (*ExecutorResourceRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *ExecutorResourceRequest) GetResourceName() string {\n\tif x != nil {\n\t\treturn x.ResourceName\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutorResourceRequest) GetAmount() int64 {\n\tif x != nil {\n\t\treturn x.Amount\n\t}\n\treturn 0\n}\n\nfunc (x *ExecutorResourceRequest) GetDiscoveryScript() string {\n\tif x != nil && x.DiscoveryScript != nil {\n\t\treturn *x.DiscoveryScript\n\t}\n\treturn \"\"\n}\n\nfunc (x *ExecutorResourceRequest) GetVendor() string {\n\tif x != nil && x.Vendor != nil {\n\t\treturn *x.Vendor\n\t}\n\treturn \"\"\n}\n\n// A task resource request.\ntype TaskResourceRequest struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) resource name.\n\tResourceName string `protobuf:\"bytes,1,opt,name=resource_name,json=resourceName,proto3\" json:\"resource_name,omitempty\"`\n\t// (Required) resource amount requesting as a double to support fractional\n\t// resource requests.\n\tAmount float64 `protobuf:\"fixed64,2,opt,name=amount,proto3\" json:\"amount,omitempty\"`\n}\n\nfunc (x *TaskResourceRequest) Reset() {\n\t*x = TaskResourceRequest{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *TaskResourceRequest) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*TaskResourceRequest) ProtoMessage() {}\n\nfunc (x *TaskResourceRequest) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use TaskResourceRequest.ProtoReflect.Descriptor instead.\nfunc (*TaskResourceRequest) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *TaskResourceRequest) GetResourceName() string {\n\tif x != nil {\n\t\treturn x.ResourceName\n\t}\n\treturn \"\"\n}\n\nfunc (x *TaskResourceRequest) GetAmount() float64 {\n\tif x != nil {\n\t\treturn x.Amount\n\t}\n\treturn 0\n}\n\ntype ResourceProfile struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Resource requests for executors. Mapped from the resource name\n\t// (e.g., cores, memory, CPU) to its specific request.\n\tExecutorResources map[string]*ExecutorResourceRequest `protobuf:\"bytes,1,rep,name=executor_resources,json=executorResources,proto3\" json:\"executor_resources,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) Resource requests for tasks. Mapped from the resource name\n\t// (e.g., cores, memory, CPU) to its specific request.\n\tTaskResources map[string]*TaskResourceRequest `protobuf:\"bytes,2,rep,name=task_resources,json=taskResources,proto3\" json:\"task_resources,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *ResourceProfile) Reset() {\n\t*x = ResourceProfile{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ResourceProfile) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ResourceProfile) ProtoMessage() {}\n\nfunc (x *ResourceProfile) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ResourceProfile.ProtoReflect.Descriptor instead.\nfunc (*ResourceProfile) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *ResourceProfile) GetExecutorResources() map[string]*ExecutorResourceRequest {\n\tif x != nil {\n\t\treturn x.ExecutorResources\n\t}\n\treturn nil\n}\n\nfunc (x *ResourceProfile) GetTaskResources() map[string]*TaskResourceRequest {\n\tif x != nil {\n\t\treturn x.TaskResources\n\t}\n\treturn nil\n}\n\ntype Origin struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Indicate the origin type.\n\t//\n\t// Types that are assignable to Function:\n\t//\n\t//\t*Origin_PythonOrigin\n\t//\t*Origin_JvmOrigin\n\tFunction isOrigin_Function `protobuf_oneof:\"function\"`\n}\n\nfunc (x *Origin) Reset() {\n\t*x = Origin{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Origin) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Origin) ProtoMessage() {}\n\nfunc (x *Origin) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Origin.ProtoReflect.Descriptor instead.\nfunc (*Origin) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (m *Origin) GetFunction() isOrigin_Function {\n\tif m != nil {\n\t\treturn m.Function\n\t}\n\treturn nil\n}\n\nfunc (x *Origin) GetPythonOrigin() *PythonOrigin {\n\tif x, ok := x.GetFunction().(*Origin_PythonOrigin); ok {\n\t\treturn x.PythonOrigin\n\t}\n\treturn nil\n}\n\nfunc (x *Origin) GetJvmOrigin() *JvmOrigin {\n\tif x, ok := x.GetFunction().(*Origin_JvmOrigin); ok {\n\t\treturn x.JvmOrigin\n\t}\n\treturn nil\n}\n\ntype isOrigin_Function interface {\n\tisOrigin_Function()\n}\n\ntype Origin_PythonOrigin struct {\n\tPythonOrigin *PythonOrigin `protobuf:\"bytes,1,opt,name=python_origin,json=pythonOrigin,proto3,oneof\"`\n}\n\ntype Origin_JvmOrigin struct {\n\tJvmOrigin *JvmOrigin `protobuf:\"bytes,2,opt,name=jvm_origin,json=jvmOrigin,proto3,oneof\"`\n}\n\nfunc (*Origin_PythonOrigin) isOrigin_Function() {}\n\nfunc (*Origin_JvmOrigin) isOrigin_Function() {}\n\ntype PythonOrigin struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Name of the origin, for example, the name of the function\n\tFragment string `protobuf:\"bytes,1,opt,name=fragment,proto3\" json:\"fragment,omitempty\"`\n\t// (Required) Callsite to show to end users, for example, stacktrace.\n\tCallSite string `protobuf:\"bytes,2,opt,name=call_site,json=callSite,proto3\" json:\"call_site,omitempty\"`\n}\n\nfunc (x *PythonOrigin) Reset() {\n\t*x = PythonOrigin{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PythonOrigin) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PythonOrigin) ProtoMessage() {}\n\nfunc (x *PythonOrigin) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PythonOrigin.ProtoReflect.Descriptor instead.\nfunc (*PythonOrigin) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (x *PythonOrigin) GetFragment() string {\n\tif x != nil {\n\t\treturn x.Fragment\n\t}\n\treturn \"\"\n}\n\nfunc (x *PythonOrigin) GetCallSite() string {\n\tif x != nil {\n\t\treturn x.CallSite\n\t}\n\treturn \"\"\n}\n\ntype JvmOrigin struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Line number in the source file.\n\tLine *int32 `protobuf:\"varint,1,opt,name=line,proto3,oneof\" json:\"line,omitempty\"`\n\t// (Optional) Start position in the source file.\n\tStartPosition *int32 `protobuf:\"varint,2,opt,name=start_position,json=startPosition,proto3,oneof\" json:\"start_position,omitempty\"`\n\t// (Optional) Start index in the source file.\n\tStartIndex *int32 `protobuf:\"varint,3,opt,name=start_index,json=startIndex,proto3,oneof\" json:\"start_index,omitempty\"`\n\t// (Optional) Stop index in the source file.\n\tStopIndex *int32 `protobuf:\"varint,4,opt,name=stop_index,json=stopIndex,proto3,oneof\" json:\"stop_index,omitempty\"`\n\t// (Optional) SQL text.\n\tSqlText *string `protobuf:\"bytes,5,opt,name=sql_text,json=sqlText,proto3,oneof\" json:\"sql_text,omitempty\"`\n\t// (Optional) Object type.\n\tObjectType *string `protobuf:\"bytes,6,opt,name=object_type,json=objectType,proto3,oneof\" json:\"object_type,omitempty\"`\n\t// (Optional) Object name.\n\tObjectName *string `protobuf:\"bytes,7,opt,name=object_name,json=objectName,proto3,oneof\" json:\"object_name,omitempty\"`\n\t// (Optional) Stack trace.\n\tStackTrace []*StackTraceElement `protobuf:\"bytes,8,rep,name=stack_trace,json=stackTrace,proto3\" json:\"stack_trace,omitempty\"`\n}\n\nfunc (x *JvmOrigin) Reset() {\n\t*x = JvmOrigin{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *JvmOrigin) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*JvmOrigin) ProtoMessage() {}\n\nfunc (x *JvmOrigin) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use JvmOrigin.ProtoReflect.Descriptor instead.\nfunc (*JvmOrigin) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (x *JvmOrigin) GetLine() int32 {\n\tif x != nil && x.Line != nil {\n\t\treturn *x.Line\n\t}\n\treturn 0\n}\n\nfunc (x *JvmOrigin) GetStartPosition() int32 {\n\tif x != nil && x.StartPosition != nil {\n\t\treturn *x.StartPosition\n\t}\n\treturn 0\n}\n\nfunc (x *JvmOrigin) GetStartIndex() int32 {\n\tif x != nil && x.StartIndex != nil {\n\t\treturn *x.StartIndex\n\t}\n\treturn 0\n}\n\nfunc (x *JvmOrigin) GetStopIndex() int32 {\n\tif x != nil && x.StopIndex != nil {\n\t\treturn *x.StopIndex\n\t}\n\treturn 0\n}\n\nfunc (x *JvmOrigin) GetSqlText() string {\n\tif x != nil && x.SqlText != nil {\n\t\treturn *x.SqlText\n\t}\n\treturn \"\"\n}\n\nfunc (x *JvmOrigin) GetObjectType() string {\n\tif x != nil && x.ObjectType != nil {\n\t\treturn *x.ObjectType\n\t}\n\treturn \"\"\n}\n\nfunc (x *JvmOrigin) GetObjectName() string {\n\tif x != nil && x.ObjectName != nil {\n\t\treturn *x.ObjectName\n\t}\n\treturn \"\"\n}\n\nfunc (x *JvmOrigin) GetStackTrace() []*StackTraceElement {\n\tif x != nil {\n\t\treturn x.StackTrace\n\t}\n\treturn nil\n}\n\n// A message to hold a [[java.lang.StackTraceElement]].\ntype StackTraceElement struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Class loader name\n\tClassLoaderName *string `protobuf:\"bytes,1,opt,name=class_loader_name,json=classLoaderName,proto3,oneof\" json:\"class_loader_name,omitempty\"`\n\t// (Optional) Module name\n\tModuleName *string `protobuf:\"bytes,2,opt,name=module_name,json=moduleName,proto3,oneof\" json:\"module_name,omitempty\"`\n\t// (Optional) Module version\n\tModuleVersion *string `protobuf:\"bytes,3,opt,name=module_version,json=moduleVersion,proto3,oneof\" json:\"module_version,omitempty\"`\n\t// (Required) Declaring class\n\tDeclaringClass string `protobuf:\"bytes,4,opt,name=declaring_class,json=declaringClass,proto3\" json:\"declaring_class,omitempty\"`\n\t// (Required) Method name\n\tMethodName string `protobuf:\"bytes,5,opt,name=method_name,json=methodName,proto3\" json:\"method_name,omitempty\"`\n\t// (Optional) File name\n\tFileName *string `protobuf:\"bytes,6,opt,name=file_name,json=fileName,proto3,oneof\" json:\"file_name,omitempty\"`\n\t// (Required) Line number\n\tLineNumber int32 `protobuf:\"varint,7,opt,name=line_number,json=lineNumber,proto3\" json:\"line_number,omitempty\"`\n}\n\nfunc (x *StackTraceElement) Reset() {\n\t*x = StackTraceElement{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StackTraceElement) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StackTraceElement) ProtoMessage() {}\n\nfunc (x *StackTraceElement) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StackTraceElement.ProtoReflect.Descriptor instead.\nfunc (*StackTraceElement) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *StackTraceElement) GetClassLoaderName() string {\n\tif x != nil && x.ClassLoaderName != nil {\n\t\treturn *x.ClassLoaderName\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetModuleName() string {\n\tif x != nil && x.ModuleName != nil {\n\t\treturn *x.ModuleName\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetModuleVersion() string {\n\tif x != nil && x.ModuleVersion != nil {\n\t\treturn *x.ModuleVersion\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetDeclaringClass() string {\n\tif x != nil {\n\t\treturn x.DeclaringClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetMethodName() string {\n\tif x != nil {\n\t\treturn x.MethodName\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetFileName() string {\n\tif x != nil && x.FileName != nil {\n\t\treturn *x.FileName\n\t}\n\treturn \"\"\n}\n\nfunc (x *StackTraceElement) GetLineNumber() int32 {\n\tif x != nil {\n\t\treturn x.LineNumber\n\t}\n\treturn 0\n}\n\ntype Bools struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []bool `protobuf:\"varint,1,rep,packed,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Bools) Reset() {\n\t*x = Bools{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Bools) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Bools) ProtoMessage() {}\n\nfunc (x *Bools) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Bools.ProtoReflect.Descriptor instead.\nfunc (*Bools) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *Bools) GetValues() []bool {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Ints struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []int32 `protobuf:\"varint,1,rep,packed,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Ints) Reset() {\n\t*x = Ints{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Ints) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Ints) ProtoMessage() {}\n\nfunc (x *Ints) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Ints.ProtoReflect.Descriptor instead.\nfunc (*Ints) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *Ints) GetValues() []int32 {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Longs struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []int64 `protobuf:\"varint,1,rep,packed,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Longs) Reset() {\n\t*x = Longs{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Longs) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Longs) ProtoMessage() {}\n\nfunc (x *Longs) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Longs.ProtoReflect.Descriptor instead.\nfunc (*Longs) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{11}\n}\n\nfunc (x *Longs) GetValues() []int64 {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Floats struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []float32 `protobuf:\"fixed32,1,rep,packed,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Floats) Reset() {\n\t*x = Floats{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Floats) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Floats) ProtoMessage() {}\n\nfunc (x *Floats) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Floats.ProtoReflect.Descriptor instead.\nfunc (*Floats) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{12}\n}\n\nfunc (x *Floats) GetValues() []float32 {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Doubles struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []float64 `protobuf:\"fixed64,1,rep,packed,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Doubles) Reset() {\n\t*x = Doubles{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Doubles) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Doubles) ProtoMessage() {}\n\nfunc (x *Doubles) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Doubles.ProtoReflect.Descriptor instead.\nfunc (*Doubles) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{13}\n}\n\nfunc (x *Doubles) GetValues() []float64 {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Strings struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []string `protobuf:\"bytes,1,rep,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Strings) Reset() {\n\t*x = Strings{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_common_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Strings) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Strings) ProtoMessage() {}\n\nfunc (x *Strings) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_common_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Strings.ProtoReflect.Descriptor instead.\nfunc (*Strings) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_common_proto_rawDescGZIP(), []int{14}\n}\n\nfunc (x *Strings) GetValues() []string {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\nvar File_spark_connect_common_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_common_proto_rawDesc = []byte{\n\t0x0a, 0x1a, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x22, 0xb0, 0x01, 0x0a, 0x0c,\n\t0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x19, 0x0a, 0x08,\n\t0x75, 0x73, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07,\n\t0x75, 0x73, 0x65, 0x44, 0x69, 0x73, 0x6b, 0x12, 0x1d, 0x0a, 0x0a, 0x75, 0x73, 0x65, 0x5f, 0x6d,\n\t0x65, 0x6d, 0x6f, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x75, 0x73, 0x65,\n\t0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x73, 0x65, 0x5f, 0x6f, 0x66,\n\t0x66, 0x5f, 0x68, 0x65, 0x61, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x75, 0x73,\n\t0x65, 0x4f, 0x66, 0x66, 0x48, 0x65, 0x61, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x64, 0x65, 0x73, 0x65,\n\t0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c,\n\t0x64, 0x65, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x12, 0x20, 0x0a, 0x0b,\n\t0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28,\n\t0x05, 0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x47,\n\t0x0a, 0x13, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x64, 0x64,\n\t0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x61, 0x64,\n\t0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x22, 0xc3, 0x01, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63,\n\t0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75,\n\t0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x6d, 0x6f, 0x75,\n\t0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74,\n\t0x12, 0x2e, 0x0a, 0x10, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x63,\n\t0x72, 0x69, 0x70, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x64, 0x69,\n\t0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x53, 0x63, 0x72, 0x69, 0x70, 0x74, 0x88, 0x01, 0x01,\n\t0x12, 0x1b, 0x0a, 0x06, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x01, 0x52, 0x06, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x42, 0x13, 0x0a,\n\t0x11, 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x5f, 0x73, 0x63, 0x72, 0x69,\n\t0x70, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x76, 0x65, 0x6e, 0x64, 0x6f, 0x72, 0x22, 0x52, 0x0a,\n\t0x13, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71,\n\t0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,\n\t0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73,\n\t0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x6d, 0x6f,\n\t0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e,\n\t0x74, 0x22, 0xa5, 0x03, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72,\n\t0x6f, 0x66, 0x69, 0x6c, 0x65, 0x12, 0x64, 0x0a, 0x12, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f,\n\t0x72, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x35, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x66, 0x69, 0x6c,\n\t0x65, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,\n\t0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x11, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,\n\t0x6f, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x58, 0x0a, 0x0e, 0x74,\n\t0x61, 0x73, 0x6b, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x66,\n\t0x69, 0x6c, 0x65, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,\n\t0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x74, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x6c, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f,\n\t0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,\n\t0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,\n\t0x79, 0x12, 0x3c, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,\n\t0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,\n\t0x02, 0x38, 0x01, 0x1a, 0x64, 0x0a, 0x12, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x6f, 0x75,\n\t0x72, 0x63, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x61, 0x73, 0x6b, 0x52,\n\t0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x05,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x93, 0x01, 0x0a, 0x06, 0x4f, 0x72,\n\t0x69, 0x67, 0x69, 0x6e, 0x12, 0x42, 0x0a, 0x0d, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x6f,\n\t0x72, 0x69, 0x67, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74, 0x68,\n\t0x6f, 0x6e, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x0c, 0x70, 0x79, 0x74, 0x68,\n\t0x6f, 0x6e, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x12, 0x39, 0x0a, 0x0a, 0x6a, 0x76, 0x6d, 0x5f,\n\t0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4a, 0x76, 0x6d,\n\t0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x09, 0x6a, 0x76, 0x6d, 0x4f, 0x72, 0x69,\n\t0x67, 0x69, 0x6e, 0x42, 0x0a, 0x0a, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22,\n\t0x47, 0x0a, 0x0c, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x12,\n\t0x1a, 0x0a, 0x08, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x08, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x63,\n\t0x61, 0x6c, 0x6c, 0x5f, 0x73, 0x69, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,\n\t0x63, 0x61, 0x6c, 0x6c, 0x53, 0x69, 0x74, 0x65, 0x22, 0xb1, 0x03, 0x0a, 0x09, 0x4a, 0x76, 0x6d,\n\t0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x12, 0x17, 0x0a, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x04, 0x6c, 0x69, 0x6e, 0x65, 0x88, 0x01, 0x01, 0x12,\n\t0x2a, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f,\n\t0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x72, 0x74,\n\t0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x73,\n\t0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05,\n\t0x48, 0x02, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x88, 0x01,\n\t0x01, 0x12, 0x22, 0x0a, 0x0a, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18,\n\t0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x03, 0x52, 0x09, 0x73, 0x74, 0x6f, 0x70, 0x49, 0x6e, 0x64,\n\t0x65, 0x78, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08, 0x73, 0x71, 0x6c, 0x5f, 0x74, 0x65, 0x78,\n\t0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x04, 0x52, 0x07, 0x73, 0x71, 0x6c, 0x54, 0x65,\n\t0x78, 0x74, 0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x48, 0x05, 0x52, 0x0a, 0x6f, 0x62,\n\t0x6a, 0x65, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x6f,\n\t0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x06, 0x52, 0x0a, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01,\n\t0x01, 0x12, 0x41, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65,\n\t0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63,\n\t0x65, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x63, 0x6b, 0x54,\n\t0x72, 0x61, 0x63, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x6c, 0x69, 0x6e, 0x65, 0x42, 0x11, 0x0a,\n\t0x0f, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e,\n\t0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78,\n\t0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42,\n\t0x0b, 0x0a, 0x09, 0x5f, 0x73, 0x71, 0x6c, 0x5f, 0x74, 0x65, 0x78, 0x74, 0x42, 0x0e, 0x0a, 0x0c,\n\t0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42, 0x0e, 0x0a, 0x0c,\n\t0x5f, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xea, 0x02, 0x0a,\n\t0x11, 0x53, 0x74, 0x61, 0x63, 0x6b, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x6c, 0x65, 0x6d, 0x65,\n\t0x6e, 0x74, 0x12, 0x2f, 0x0a, 0x11, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6c, 0x6f, 0x61, 0x64,\n\t0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,\n\t0x0f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65,\n\t0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x6d, 0x6f, 0x64, 0x75, 0x6c, 0x65, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0a, 0x6d, 0x6f, 0x64, 0x75,\n\t0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x2a, 0x0a, 0x0e, 0x6d, 0x6f, 0x64,\n\t0x75, 0x6c, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x02, 0x52, 0x0d, 0x6d, 0x6f, 0x64, 0x75, 0x6c, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69,\n\t0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x27, 0x0a, 0x0f, 0x64, 0x65, 0x63, 0x6c, 0x61, 0x72, 0x69,\n\t0x6e, 0x67, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,\n\t0x64, 0x65, 0x63, 0x6c, 0x61, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x12, 0x1f,\n\t0x0a, 0x0b, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12,\n\t0x20, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01,\n\t0x01, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72,\n\t0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x6c, 0x69, 0x6e, 0x65, 0x4e, 0x75, 0x6d, 0x62,\n\t0x65, 0x72, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6c, 0x6f, 0x61,\n\t0x64, 0x65, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x6d, 0x6f, 0x64,\n\t0x75, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x6f, 0x64,\n\t0x75, 0x6c, 0x65, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x0c, 0x0a, 0x0a, 0x5f,\n\t0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x1f, 0x0a, 0x05, 0x42, 0x6f, 0x6f,\n\t0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03,\n\t0x28, 0x08, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x1e, 0x0a, 0x04, 0x49, 0x6e,\n\t0x74, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03,\n\t0x28, 0x05, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x1f, 0x0a, 0x05, 0x4c, 0x6f,\n\t0x6e, 0x67, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20,\n\t0x03, 0x28, 0x03, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x20, 0x0a, 0x06, 0x46,\n\t0x6c, 0x6f, 0x61, 0x74, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18,\n\t0x01, 0x20, 0x03, 0x28, 0x02, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x21, 0x0a,\n\t0x07, 0x44, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75,\n\t0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x01, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73,\n\t0x22, 0x21, 0x0a, 0x07, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x73, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,\n\t0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,\n\t0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_common_proto_rawDescOnce sync.Once\n\tfile_spark_connect_common_proto_rawDescData = file_spark_connect_common_proto_rawDesc\n)\n\nfunc file_spark_connect_common_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_common_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_common_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_common_proto_rawDescData)\n\t})\n\treturn file_spark_connect_common_proto_rawDescData\n}\n\nvar file_spark_connect_common_proto_msgTypes = make([]protoimpl.MessageInfo, 17)\nvar file_spark_connect_common_proto_goTypes = []interface{}{\n\t(*StorageLevel)(nil),            // 0: spark.connect.StorageLevel\n\t(*ResourceInformation)(nil),     // 1: spark.connect.ResourceInformation\n\t(*ExecutorResourceRequest)(nil), // 2: spark.connect.ExecutorResourceRequest\n\t(*TaskResourceRequest)(nil),     // 3: spark.connect.TaskResourceRequest\n\t(*ResourceProfile)(nil),         // 4: spark.connect.ResourceProfile\n\t(*Origin)(nil),                  // 5: spark.connect.Origin\n\t(*PythonOrigin)(nil),            // 6: spark.connect.PythonOrigin\n\t(*JvmOrigin)(nil),               // 7: spark.connect.JvmOrigin\n\t(*StackTraceElement)(nil),       // 8: spark.connect.StackTraceElement\n\t(*Bools)(nil),                   // 9: spark.connect.Bools\n\t(*Ints)(nil),                    // 10: spark.connect.Ints\n\t(*Longs)(nil),                   // 11: spark.connect.Longs\n\t(*Floats)(nil),                  // 12: spark.connect.Floats\n\t(*Doubles)(nil),                 // 13: spark.connect.Doubles\n\t(*Strings)(nil),                 // 14: spark.connect.Strings\n\tnil,                             // 15: spark.connect.ResourceProfile.ExecutorResourcesEntry\n\tnil,                             // 16: spark.connect.ResourceProfile.TaskResourcesEntry\n}\nvar file_spark_connect_common_proto_depIdxs = []int32{\n\t15, // 0: spark.connect.ResourceProfile.executor_resources:type_name -> spark.connect.ResourceProfile.ExecutorResourcesEntry\n\t16, // 1: spark.connect.ResourceProfile.task_resources:type_name -> spark.connect.ResourceProfile.TaskResourcesEntry\n\t6,  // 2: spark.connect.Origin.python_origin:type_name -> spark.connect.PythonOrigin\n\t7,  // 3: spark.connect.Origin.jvm_origin:type_name -> spark.connect.JvmOrigin\n\t8,  // 4: spark.connect.JvmOrigin.stack_trace:type_name -> spark.connect.StackTraceElement\n\t2,  // 5: spark.connect.ResourceProfile.ExecutorResourcesEntry.value:type_name -> spark.connect.ExecutorResourceRequest\n\t3,  // 6: spark.connect.ResourceProfile.TaskResourcesEntry.value:type_name -> spark.connect.TaskResourceRequest\n\t7,  // [7:7] is the sub-list for method output_type\n\t7,  // [7:7] is the sub-list for method input_type\n\t7,  // [7:7] is the sub-list for extension type_name\n\t7,  // [7:7] is the sub-list for extension extendee\n\t0,  // [0:7] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_common_proto_init() }\nfunc file_spark_connect_common_proto_init() {\n\tif File_spark_connect_common_proto != nil {\n\t\treturn\n\t}\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_common_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StorageLevel); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ResourceInformation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExecutorResourceRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*TaskResourceRequest); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ResourceProfile); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Origin); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PythonOrigin); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*JvmOrigin); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StackTraceElement); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Bools); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Ints); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Longs); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Floats); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Doubles); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_common_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Strings); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_common_proto_msgTypes[2].OneofWrappers = []interface{}{}\n\tfile_spark_connect_common_proto_msgTypes[5].OneofWrappers = []interface{}{\n\t\t(*Origin_PythonOrigin)(nil),\n\t\t(*Origin_JvmOrigin)(nil),\n\t}\n\tfile_spark_connect_common_proto_msgTypes[7].OneofWrappers = []interface{}{}\n\tfile_spark_connect_common_proto_msgTypes[8].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_common_proto_rawDesc,\n\t\t\tNumEnums:      0,\n\t\t\tNumMessages:   17,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_common_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_common_proto_depIdxs,\n\t\tMessageInfos:      file_spark_connect_common_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_common_proto = out.File\n\tfile_spark_connect_common_proto_rawDesc = nil\n\tfile_spark_connect_common_proto_goTypes = nil\n\tfile_spark_connect_common_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/example_plugins.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/example_plugins.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\ntype ExamplePluginRelation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tInput       *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\tCustomField string    `protobuf:\"bytes,2,opt,name=custom_field,json=customField,proto3\" json:\"custom_field,omitempty\"`\n}\n\nfunc (x *ExamplePluginRelation) Reset() {\n\t*x = ExamplePluginRelation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_example_plugins_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExamplePluginRelation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExamplePluginRelation) ProtoMessage() {}\n\nfunc (x *ExamplePluginRelation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_example_plugins_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExamplePluginRelation.ProtoReflect.Descriptor instead.\nfunc (*ExamplePluginRelation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_example_plugins_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (x *ExamplePluginRelation) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *ExamplePluginRelation) GetCustomField() string {\n\tif x != nil {\n\t\treturn x.CustomField\n\t}\n\treturn \"\"\n}\n\ntype ExamplePluginExpression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tChild       *Expression `protobuf:\"bytes,1,opt,name=child,proto3\" json:\"child,omitempty\"`\n\tCustomField string      `protobuf:\"bytes,2,opt,name=custom_field,json=customField,proto3\" json:\"custom_field,omitempty\"`\n}\n\nfunc (x *ExamplePluginExpression) Reset() {\n\t*x = ExamplePluginExpression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_example_plugins_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExamplePluginExpression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExamplePluginExpression) ProtoMessage() {}\n\nfunc (x *ExamplePluginExpression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_example_plugins_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExamplePluginExpression.ProtoReflect.Descriptor instead.\nfunc (*ExamplePluginExpression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_example_plugins_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *ExamplePluginExpression) GetChild() *Expression {\n\tif x != nil {\n\t\treturn x.Child\n\t}\n\treturn nil\n}\n\nfunc (x *ExamplePluginExpression) GetCustomField() string {\n\tif x != nil {\n\t\treturn x.CustomField\n\t}\n\treturn \"\"\n}\n\ntype ExamplePluginCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tCustomField string `protobuf:\"bytes,1,opt,name=custom_field,json=customField,proto3\" json:\"custom_field,omitempty\"`\n}\n\nfunc (x *ExamplePluginCommand) Reset() {\n\t*x = ExamplePluginCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_example_plugins_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExamplePluginCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExamplePluginCommand) ProtoMessage() {}\n\nfunc (x *ExamplePluginCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_example_plugins_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExamplePluginCommand.ProtoReflect.Descriptor instead.\nfunc (*ExamplePluginCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_example_plugins_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *ExamplePluginCommand) GetCustomField() string {\n\tif x != nil {\n\t\treturn x.CustomField\n\t}\n\treturn \"\"\n}\n\nvar File_spark_connect_example_plugins_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_example_plugins_proto_rawDesc = []byte{\n\t0x0a, 0x23, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x65, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x73, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x1a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72,\n\t0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70,\n\t0x72, 0x6f, 0x74, 0x6f, 0x22, 0x69, 0x0a, 0x15, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50,\n\t0x6c, 0x75, 0x67, 0x69, 0x6e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c,\n\t0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x0b, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x22,\n\t0x6d, 0x0a, 0x17, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x05, 0x63, 0x68,\n\t0x69, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63,\n\t0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x0b, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x22, 0x39,\n\t0x0a, 0x14, 0x45, 0x78, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43,\n\t0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d,\n\t0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x75,\n\t0x73, 0x74, 0x6f, 0x6d, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67,\n\t0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69,\n\t0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65,\n\t0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_example_plugins_proto_rawDescOnce sync.Once\n\tfile_spark_connect_example_plugins_proto_rawDescData = file_spark_connect_example_plugins_proto_rawDesc\n)\n\nfunc file_spark_connect_example_plugins_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_example_plugins_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_example_plugins_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_example_plugins_proto_rawDescData)\n\t})\n\treturn file_spark_connect_example_plugins_proto_rawDescData\n}\n\nvar file_spark_connect_example_plugins_proto_msgTypes = make([]protoimpl.MessageInfo, 3)\nvar file_spark_connect_example_plugins_proto_goTypes = []interface{}{\n\t(*ExamplePluginRelation)(nil),   // 0: spark.connect.ExamplePluginRelation\n\t(*ExamplePluginExpression)(nil), // 1: spark.connect.ExamplePluginExpression\n\t(*ExamplePluginCommand)(nil),    // 2: spark.connect.ExamplePluginCommand\n\t(*Relation)(nil),                // 3: spark.connect.Relation\n\t(*Expression)(nil),              // 4: spark.connect.Expression\n}\nvar file_spark_connect_example_plugins_proto_depIdxs = []int32{\n\t3, // 0: spark.connect.ExamplePluginRelation.input:type_name -> spark.connect.Relation\n\t4, // 1: spark.connect.ExamplePluginExpression.child:type_name -> spark.connect.Expression\n\t2, // [2:2] is the sub-list for method output_type\n\t2, // [2:2] is the sub-list for method input_type\n\t2, // [2:2] is the sub-list for extension type_name\n\t2, // [2:2] is the sub-list for extension extendee\n\t0, // [0:2] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_example_plugins_proto_init() }\nfunc file_spark_connect_example_plugins_proto_init() {\n\tif File_spark_connect_example_plugins_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_relations_proto_init()\n\tfile_spark_connect_expressions_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_example_plugins_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExamplePluginRelation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_example_plugins_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExamplePluginExpression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_example_plugins_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExamplePluginCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_example_plugins_proto_rawDesc,\n\t\t\tNumEnums:      0,\n\t\t\tNumMessages:   3,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_example_plugins_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_example_plugins_proto_depIdxs,\n\t\tMessageInfos:      file_spark_connect_example_plugins_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_example_plugins_proto = out.File\n\tfile_spark_connect_example_plugins_proto_rawDesc = nil\n\tfile_spark_connect_example_plugins_proto_goTypes = nil\n\tfile_spark_connect_example_plugins_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/expressions.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/expressions.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\tanypb \"google.golang.org/protobuf/types/known/anypb\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\ntype Expression_Window_WindowFrame_FrameType int32\n\nconst (\n\tExpression_Window_WindowFrame_FRAME_TYPE_UNDEFINED Expression_Window_WindowFrame_FrameType = 0\n\t// RowFrame treats rows in a partition individually.\n\tExpression_Window_WindowFrame_FRAME_TYPE_ROW Expression_Window_WindowFrame_FrameType = 1\n\t// RangeFrame treats rows in a partition as groups of peers.\n\t// All rows having the same 'ORDER BY' ordering are considered as peers.\n\tExpression_Window_WindowFrame_FRAME_TYPE_RANGE Expression_Window_WindowFrame_FrameType = 2\n)\n\n// Enum value maps for Expression_Window_WindowFrame_FrameType.\nvar (\n\tExpression_Window_WindowFrame_FrameType_name = map[int32]string{\n\t\t0: \"FRAME_TYPE_UNDEFINED\",\n\t\t1: \"FRAME_TYPE_ROW\",\n\t\t2: \"FRAME_TYPE_RANGE\",\n\t}\n\tExpression_Window_WindowFrame_FrameType_value = map[string]int32{\n\t\t\"FRAME_TYPE_UNDEFINED\": 0,\n\t\t\"FRAME_TYPE_ROW\":       1,\n\t\t\"FRAME_TYPE_RANGE\":     2,\n\t}\n)\n\nfunc (x Expression_Window_WindowFrame_FrameType) Enum() *Expression_Window_WindowFrame_FrameType {\n\tp := new(Expression_Window_WindowFrame_FrameType)\n\t*p = x\n\treturn p\n}\n\nfunc (x Expression_Window_WindowFrame_FrameType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Expression_Window_WindowFrame_FrameType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[0].Descriptor()\n}\n\nfunc (Expression_Window_WindowFrame_FrameType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[0]\n}\n\nfunc (x Expression_Window_WindowFrame_FrameType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Expression_Window_WindowFrame_FrameType.Descriptor instead.\nfunc (Expression_Window_WindowFrame_FrameType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 0, 0, 0}\n}\n\ntype Expression_SortOrder_SortDirection int32\n\nconst (\n\tExpression_SortOrder_SORT_DIRECTION_UNSPECIFIED Expression_SortOrder_SortDirection = 0\n\tExpression_SortOrder_SORT_DIRECTION_ASCENDING   Expression_SortOrder_SortDirection = 1\n\tExpression_SortOrder_SORT_DIRECTION_DESCENDING  Expression_SortOrder_SortDirection = 2\n)\n\n// Enum value maps for Expression_SortOrder_SortDirection.\nvar (\n\tExpression_SortOrder_SortDirection_name = map[int32]string{\n\t\t0: \"SORT_DIRECTION_UNSPECIFIED\",\n\t\t1: \"SORT_DIRECTION_ASCENDING\",\n\t\t2: \"SORT_DIRECTION_DESCENDING\",\n\t}\n\tExpression_SortOrder_SortDirection_value = map[string]int32{\n\t\t\"SORT_DIRECTION_UNSPECIFIED\": 0,\n\t\t\"SORT_DIRECTION_ASCENDING\":   1,\n\t\t\"SORT_DIRECTION_DESCENDING\":  2,\n\t}\n)\n\nfunc (x Expression_SortOrder_SortDirection) Enum() *Expression_SortOrder_SortDirection {\n\tp := new(Expression_SortOrder_SortDirection)\n\t*p = x\n\treturn p\n}\n\nfunc (x Expression_SortOrder_SortDirection) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Expression_SortOrder_SortDirection) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[1].Descriptor()\n}\n\nfunc (Expression_SortOrder_SortDirection) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[1]\n}\n\nfunc (x Expression_SortOrder_SortDirection) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Expression_SortOrder_SortDirection.Descriptor instead.\nfunc (Expression_SortOrder_SortDirection) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 1, 0}\n}\n\ntype Expression_SortOrder_NullOrdering int32\n\nconst (\n\tExpression_SortOrder_SORT_NULLS_UNSPECIFIED Expression_SortOrder_NullOrdering = 0\n\tExpression_SortOrder_SORT_NULLS_FIRST       Expression_SortOrder_NullOrdering = 1\n\tExpression_SortOrder_SORT_NULLS_LAST        Expression_SortOrder_NullOrdering = 2\n)\n\n// Enum value maps for Expression_SortOrder_NullOrdering.\nvar (\n\tExpression_SortOrder_NullOrdering_name = map[int32]string{\n\t\t0: \"SORT_NULLS_UNSPECIFIED\",\n\t\t1: \"SORT_NULLS_FIRST\",\n\t\t2: \"SORT_NULLS_LAST\",\n\t}\n\tExpression_SortOrder_NullOrdering_value = map[string]int32{\n\t\t\"SORT_NULLS_UNSPECIFIED\": 0,\n\t\t\"SORT_NULLS_FIRST\":       1,\n\t\t\"SORT_NULLS_LAST\":        2,\n\t}\n)\n\nfunc (x Expression_SortOrder_NullOrdering) Enum() *Expression_SortOrder_NullOrdering {\n\tp := new(Expression_SortOrder_NullOrdering)\n\t*p = x\n\treturn p\n}\n\nfunc (x Expression_SortOrder_NullOrdering) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Expression_SortOrder_NullOrdering) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[2].Descriptor()\n}\n\nfunc (Expression_SortOrder_NullOrdering) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[2]\n}\n\nfunc (x Expression_SortOrder_NullOrdering) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Expression_SortOrder_NullOrdering.Descriptor instead.\nfunc (Expression_SortOrder_NullOrdering) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 1, 1}\n}\n\ntype Expression_Cast_EvalMode int32\n\nconst (\n\tExpression_Cast_EVAL_MODE_UNSPECIFIED Expression_Cast_EvalMode = 0\n\tExpression_Cast_EVAL_MODE_LEGACY      Expression_Cast_EvalMode = 1\n\tExpression_Cast_EVAL_MODE_ANSI        Expression_Cast_EvalMode = 2\n\tExpression_Cast_EVAL_MODE_TRY         Expression_Cast_EvalMode = 3\n)\n\n// Enum value maps for Expression_Cast_EvalMode.\nvar (\n\tExpression_Cast_EvalMode_name = map[int32]string{\n\t\t0: \"EVAL_MODE_UNSPECIFIED\",\n\t\t1: \"EVAL_MODE_LEGACY\",\n\t\t2: \"EVAL_MODE_ANSI\",\n\t\t3: \"EVAL_MODE_TRY\",\n\t}\n\tExpression_Cast_EvalMode_value = map[string]int32{\n\t\t\"EVAL_MODE_UNSPECIFIED\": 0,\n\t\t\"EVAL_MODE_LEGACY\":      1,\n\t\t\"EVAL_MODE_ANSI\":        2,\n\t\t\"EVAL_MODE_TRY\":         3,\n\t}\n)\n\nfunc (x Expression_Cast_EvalMode) Enum() *Expression_Cast_EvalMode {\n\tp := new(Expression_Cast_EvalMode)\n\t*p = x\n\treturn p\n}\n\nfunc (x Expression_Cast_EvalMode) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Expression_Cast_EvalMode) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[3].Descriptor()\n}\n\nfunc (Expression_Cast_EvalMode) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[3]\n}\n\nfunc (x Expression_Cast_EvalMode) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Expression_Cast_EvalMode.Descriptor instead.\nfunc (Expression_Cast_EvalMode) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 2, 0}\n}\n\ntype MergeAction_ActionType int32\n\nconst (\n\tMergeAction_ACTION_TYPE_INVALID     MergeAction_ActionType = 0\n\tMergeAction_ACTION_TYPE_DELETE      MergeAction_ActionType = 1\n\tMergeAction_ACTION_TYPE_INSERT      MergeAction_ActionType = 2\n\tMergeAction_ACTION_TYPE_INSERT_STAR MergeAction_ActionType = 3\n\tMergeAction_ACTION_TYPE_UPDATE      MergeAction_ActionType = 4\n\tMergeAction_ACTION_TYPE_UPDATE_STAR MergeAction_ActionType = 5\n)\n\n// Enum value maps for MergeAction_ActionType.\nvar (\n\tMergeAction_ActionType_name = map[int32]string{\n\t\t0: \"ACTION_TYPE_INVALID\",\n\t\t1: \"ACTION_TYPE_DELETE\",\n\t\t2: \"ACTION_TYPE_INSERT\",\n\t\t3: \"ACTION_TYPE_INSERT_STAR\",\n\t\t4: \"ACTION_TYPE_UPDATE\",\n\t\t5: \"ACTION_TYPE_UPDATE_STAR\",\n\t}\n\tMergeAction_ActionType_value = map[string]int32{\n\t\t\"ACTION_TYPE_INVALID\":     0,\n\t\t\"ACTION_TYPE_DELETE\":      1,\n\t\t\"ACTION_TYPE_INSERT\":      2,\n\t\t\"ACTION_TYPE_INSERT_STAR\": 3,\n\t\t\"ACTION_TYPE_UPDATE\":      4,\n\t\t\"ACTION_TYPE_UPDATE_STAR\": 5,\n\t}\n)\n\nfunc (x MergeAction_ActionType) Enum() *MergeAction_ActionType {\n\tp := new(MergeAction_ActionType)\n\t*p = x\n\treturn p\n}\n\nfunc (x MergeAction_ActionType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (MergeAction_ActionType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[4].Descriptor()\n}\n\nfunc (MergeAction_ActionType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[4]\n}\n\nfunc (x MergeAction_ActionType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use MergeAction_ActionType.Descriptor instead.\nfunc (MergeAction_ActionType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{9, 0}\n}\n\ntype SubqueryExpression_SubqueryType int32\n\nconst (\n\tSubqueryExpression_SUBQUERY_TYPE_UNKNOWN   SubqueryExpression_SubqueryType = 0\n\tSubqueryExpression_SUBQUERY_TYPE_SCALAR    SubqueryExpression_SubqueryType = 1\n\tSubqueryExpression_SUBQUERY_TYPE_EXISTS    SubqueryExpression_SubqueryType = 2\n\tSubqueryExpression_SUBQUERY_TYPE_TABLE_ARG SubqueryExpression_SubqueryType = 3\n\tSubqueryExpression_SUBQUERY_TYPE_IN        SubqueryExpression_SubqueryType = 4\n)\n\n// Enum value maps for SubqueryExpression_SubqueryType.\nvar (\n\tSubqueryExpression_SubqueryType_name = map[int32]string{\n\t\t0: \"SUBQUERY_TYPE_UNKNOWN\",\n\t\t1: \"SUBQUERY_TYPE_SCALAR\",\n\t\t2: \"SUBQUERY_TYPE_EXISTS\",\n\t\t3: \"SUBQUERY_TYPE_TABLE_ARG\",\n\t\t4: \"SUBQUERY_TYPE_IN\",\n\t}\n\tSubqueryExpression_SubqueryType_value = map[string]int32{\n\t\t\"SUBQUERY_TYPE_UNKNOWN\":   0,\n\t\t\"SUBQUERY_TYPE_SCALAR\":    1,\n\t\t\"SUBQUERY_TYPE_EXISTS\":    2,\n\t\t\"SUBQUERY_TYPE_TABLE_ARG\": 3,\n\t\t\"SUBQUERY_TYPE_IN\":        4,\n\t}\n)\n\nfunc (x SubqueryExpression_SubqueryType) Enum() *SubqueryExpression_SubqueryType {\n\tp := new(SubqueryExpression_SubqueryType)\n\t*p = x\n\treturn p\n}\n\nfunc (x SubqueryExpression_SubqueryType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (SubqueryExpression_SubqueryType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_expressions_proto_enumTypes[5].Descriptor()\n}\n\nfunc (SubqueryExpression_SubqueryType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_expressions_proto_enumTypes[5]\n}\n\nfunc (x SubqueryExpression_SubqueryType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use SubqueryExpression_SubqueryType.Descriptor instead.\nfunc (SubqueryExpression_SubqueryType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{10, 0}\n}\n\n// Expression used to refer to fields, functions and similar. This can be used everywhere\n// expressions in SQL appear.\ntype Expression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tCommon *ExpressionCommon `protobuf:\"bytes,18,opt,name=common,proto3\" json:\"common,omitempty\"`\n\t// Types that are assignable to ExprType:\n\t//\n\t//\t*Expression_Literal_\n\t//\t*Expression_UnresolvedAttribute_\n\t//\t*Expression_UnresolvedFunction_\n\t//\t*Expression_ExpressionString_\n\t//\t*Expression_UnresolvedStar_\n\t//\t*Expression_Alias_\n\t//\t*Expression_Cast_\n\t//\t*Expression_UnresolvedRegex_\n\t//\t*Expression_SortOrder_\n\t//\t*Expression_LambdaFunction_\n\t//\t*Expression_Window_\n\t//\t*Expression_UnresolvedExtractValue_\n\t//\t*Expression_UpdateFields_\n\t//\t*Expression_UnresolvedNamedLambdaVariable_\n\t//\t*Expression_CommonInlineUserDefinedFunction\n\t//\t*Expression_CallFunction\n\t//\t*Expression_NamedArgumentExpression\n\t//\t*Expression_MergeAction\n\t//\t*Expression_TypedAggregateExpression\n\t//\t*Expression_SubqueryExpression\n\t//\t*Expression_Extension\n\tExprType isExpression_ExprType `protobuf_oneof:\"expr_type\"`\n}\n\nfunc (x *Expression) Reset() {\n\t*x = Expression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression) ProtoMessage() {}\n\nfunc (x *Expression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression.ProtoReflect.Descriptor instead.\nfunc (*Expression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (x *Expression) GetCommon() *ExpressionCommon {\n\tif x != nil {\n\t\treturn x.Common\n\t}\n\treturn nil\n}\n\nfunc (m *Expression) GetExprType() isExpression_ExprType {\n\tif m != nil {\n\t\treturn m.ExprType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetLiteral() *Expression_Literal {\n\tif x, ok := x.GetExprType().(*Expression_Literal_); ok {\n\t\treturn x.Literal\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedAttribute() *Expression_UnresolvedAttribute {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedAttribute_); ok {\n\t\treturn x.UnresolvedAttribute\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedFunction() *Expression_UnresolvedFunction {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedFunction_); ok {\n\t\treturn x.UnresolvedFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetExpressionString() *Expression_ExpressionString {\n\tif x, ok := x.GetExprType().(*Expression_ExpressionString_); ok {\n\t\treturn x.ExpressionString\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedStar() *Expression_UnresolvedStar {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedStar_); ok {\n\t\treturn x.UnresolvedStar\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetAlias() *Expression_Alias {\n\tif x, ok := x.GetExprType().(*Expression_Alias_); ok {\n\t\treturn x.Alias\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetCast() *Expression_Cast {\n\tif x, ok := x.GetExprType().(*Expression_Cast_); ok {\n\t\treturn x.Cast\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedRegex() *Expression_UnresolvedRegex {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedRegex_); ok {\n\t\treturn x.UnresolvedRegex\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetSortOrder() *Expression_SortOrder {\n\tif x, ok := x.GetExprType().(*Expression_SortOrder_); ok {\n\t\treturn x.SortOrder\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetLambdaFunction() *Expression_LambdaFunction {\n\tif x, ok := x.GetExprType().(*Expression_LambdaFunction_); ok {\n\t\treturn x.LambdaFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetWindow() *Expression_Window {\n\tif x, ok := x.GetExprType().(*Expression_Window_); ok {\n\t\treturn x.Window\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedExtractValue() *Expression_UnresolvedExtractValue {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedExtractValue_); ok {\n\t\treturn x.UnresolvedExtractValue\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUpdateFields() *Expression_UpdateFields {\n\tif x, ok := x.GetExprType().(*Expression_UpdateFields_); ok {\n\t\treturn x.UpdateFields\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetUnresolvedNamedLambdaVariable() *Expression_UnresolvedNamedLambdaVariable {\n\tif x, ok := x.GetExprType().(*Expression_UnresolvedNamedLambdaVariable_); ok {\n\t\treturn x.UnresolvedNamedLambdaVariable\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetCommonInlineUserDefinedFunction() *CommonInlineUserDefinedFunction {\n\tif x, ok := x.GetExprType().(*Expression_CommonInlineUserDefinedFunction); ok {\n\t\treturn x.CommonInlineUserDefinedFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetCallFunction() *CallFunction {\n\tif x, ok := x.GetExprType().(*Expression_CallFunction); ok {\n\t\treturn x.CallFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetNamedArgumentExpression() *NamedArgumentExpression {\n\tif x, ok := x.GetExprType().(*Expression_NamedArgumentExpression); ok {\n\t\treturn x.NamedArgumentExpression\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetMergeAction() *MergeAction {\n\tif x, ok := x.GetExprType().(*Expression_MergeAction); ok {\n\t\treturn x.MergeAction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetTypedAggregateExpression() *TypedAggregateExpression {\n\tif x, ok := x.GetExprType().(*Expression_TypedAggregateExpression); ok {\n\t\treturn x.TypedAggregateExpression\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetSubqueryExpression() *SubqueryExpression {\n\tif x, ok := x.GetExprType().(*Expression_SubqueryExpression); ok {\n\t\treturn x.SubqueryExpression\n\t}\n\treturn nil\n}\n\nfunc (x *Expression) GetExtension() *anypb.Any {\n\tif x, ok := x.GetExprType().(*Expression_Extension); ok {\n\t\treturn x.Extension\n\t}\n\treturn nil\n}\n\ntype isExpression_ExprType interface {\n\tisExpression_ExprType()\n}\n\ntype Expression_Literal_ struct {\n\tLiteral *Expression_Literal `protobuf:\"bytes,1,opt,name=literal,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedAttribute_ struct {\n\tUnresolvedAttribute *Expression_UnresolvedAttribute `protobuf:\"bytes,2,opt,name=unresolved_attribute,json=unresolvedAttribute,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedFunction_ struct {\n\tUnresolvedFunction *Expression_UnresolvedFunction `protobuf:\"bytes,3,opt,name=unresolved_function,json=unresolvedFunction,proto3,oneof\"`\n}\n\ntype Expression_ExpressionString_ struct {\n\tExpressionString *Expression_ExpressionString `protobuf:\"bytes,4,opt,name=expression_string,json=expressionString,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedStar_ struct {\n\tUnresolvedStar *Expression_UnresolvedStar `protobuf:\"bytes,5,opt,name=unresolved_star,json=unresolvedStar,proto3,oneof\"`\n}\n\ntype Expression_Alias_ struct {\n\tAlias *Expression_Alias `protobuf:\"bytes,6,opt,name=alias,proto3,oneof\"`\n}\n\ntype Expression_Cast_ struct {\n\tCast *Expression_Cast `protobuf:\"bytes,7,opt,name=cast,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedRegex_ struct {\n\tUnresolvedRegex *Expression_UnresolvedRegex `protobuf:\"bytes,8,opt,name=unresolved_regex,json=unresolvedRegex,proto3,oneof\"`\n}\n\ntype Expression_SortOrder_ struct {\n\tSortOrder *Expression_SortOrder `protobuf:\"bytes,9,opt,name=sort_order,json=sortOrder,proto3,oneof\"`\n}\n\ntype Expression_LambdaFunction_ struct {\n\tLambdaFunction *Expression_LambdaFunction `protobuf:\"bytes,10,opt,name=lambda_function,json=lambdaFunction,proto3,oneof\"`\n}\n\ntype Expression_Window_ struct {\n\tWindow *Expression_Window `protobuf:\"bytes,11,opt,name=window,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedExtractValue_ struct {\n\tUnresolvedExtractValue *Expression_UnresolvedExtractValue `protobuf:\"bytes,12,opt,name=unresolved_extract_value,json=unresolvedExtractValue,proto3,oneof\"`\n}\n\ntype Expression_UpdateFields_ struct {\n\tUpdateFields *Expression_UpdateFields `protobuf:\"bytes,13,opt,name=update_fields,json=updateFields,proto3,oneof\"`\n}\n\ntype Expression_UnresolvedNamedLambdaVariable_ struct {\n\tUnresolvedNamedLambdaVariable *Expression_UnresolvedNamedLambdaVariable `protobuf:\"bytes,14,opt,name=unresolved_named_lambda_variable,json=unresolvedNamedLambdaVariable,proto3,oneof\"`\n}\n\ntype Expression_CommonInlineUserDefinedFunction struct {\n\tCommonInlineUserDefinedFunction *CommonInlineUserDefinedFunction `protobuf:\"bytes,15,opt,name=common_inline_user_defined_function,json=commonInlineUserDefinedFunction,proto3,oneof\"`\n}\n\ntype Expression_CallFunction struct {\n\tCallFunction *CallFunction `protobuf:\"bytes,16,opt,name=call_function,json=callFunction,proto3,oneof\"`\n}\n\ntype Expression_NamedArgumentExpression struct {\n\tNamedArgumentExpression *NamedArgumentExpression `protobuf:\"bytes,17,opt,name=named_argument_expression,json=namedArgumentExpression,proto3,oneof\"`\n}\n\ntype Expression_MergeAction struct {\n\tMergeAction *MergeAction `protobuf:\"bytes,19,opt,name=merge_action,json=mergeAction,proto3,oneof\"`\n}\n\ntype Expression_TypedAggregateExpression struct {\n\tTypedAggregateExpression *TypedAggregateExpression `protobuf:\"bytes,20,opt,name=typed_aggregate_expression,json=typedAggregateExpression,proto3,oneof\"`\n}\n\ntype Expression_SubqueryExpression struct {\n\tSubqueryExpression *SubqueryExpression `protobuf:\"bytes,21,opt,name=subquery_expression,json=subqueryExpression,proto3,oneof\"`\n}\n\ntype Expression_Extension struct {\n\t// This field is used to mark extensions to the protocol. When plugins generate arbitrary\n\t// relations they can add them here. During the planning the correct resolution is done.\n\tExtension *anypb.Any `protobuf:\"bytes,999,opt,name=extension,proto3,oneof\"`\n}\n\nfunc (*Expression_Literal_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedAttribute_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedFunction_) isExpression_ExprType() {}\n\nfunc (*Expression_ExpressionString_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedStar_) isExpression_ExprType() {}\n\nfunc (*Expression_Alias_) isExpression_ExprType() {}\n\nfunc (*Expression_Cast_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedRegex_) isExpression_ExprType() {}\n\nfunc (*Expression_SortOrder_) isExpression_ExprType() {}\n\nfunc (*Expression_LambdaFunction_) isExpression_ExprType() {}\n\nfunc (*Expression_Window_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedExtractValue_) isExpression_ExprType() {}\n\nfunc (*Expression_UpdateFields_) isExpression_ExprType() {}\n\nfunc (*Expression_UnresolvedNamedLambdaVariable_) isExpression_ExprType() {}\n\nfunc (*Expression_CommonInlineUserDefinedFunction) isExpression_ExprType() {}\n\nfunc (*Expression_CallFunction) isExpression_ExprType() {}\n\nfunc (*Expression_NamedArgumentExpression) isExpression_ExprType() {}\n\nfunc (*Expression_MergeAction) isExpression_ExprType() {}\n\nfunc (*Expression_TypedAggregateExpression) isExpression_ExprType() {}\n\nfunc (*Expression_SubqueryExpression) isExpression_ExprType() {}\n\nfunc (*Expression_Extension) isExpression_ExprType() {}\n\ntype ExpressionCommon struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Keep the information of the origin for this expression such as stacktrace.\n\tOrigin *Origin `protobuf:\"bytes,1,opt,name=origin,proto3\" json:\"origin,omitempty\"`\n}\n\nfunc (x *ExpressionCommon) Reset() {\n\t*x = ExpressionCommon{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ExpressionCommon) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ExpressionCommon) ProtoMessage() {}\n\nfunc (x *ExpressionCommon) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ExpressionCommon.ProtoReflect.Descriptor instead.\nfunc (*ExpressionCommon) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *ExpressionCommon) GetOrigin() *Origin {\n\tif x != nil {\n\t\treturn x.Origin\n\t}\n\treturn nil\n}\n\ntype CommonInlineUserDefinedFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Name of the user-defined function.\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional) Indicate if the user-defined function is deterministic.\n\tDeterministic bool `protobuf:\"varint,2,opt,name=deterministic,proto3\" json:\"deterministic,omitempty\"`\n\t// (Optional) Function arguments. Empty arguments are allowed.\n\tArguments []*Expression `protobuf:\"bytes,3,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n\t// (Required) Indicate the function type of the user-defined function.\n\t//\n\t// Types that are assignable to Function:\n\t//\n\t//\t*CommonInlineUserDefinedFunction_PythonUdf\n\t//\t*CommonInlineUserDefinedFunction_ScalarScalaUdf\n\t//\t*CommonInlineUserDefinedFunction_JavaUdf\n\tFunction isCommonInlineUserDefinedFunction_Function `protobuf_oneof:\"function\"`\n\t// (Required) Indicate if this function should be applied on distinct values.\n\tIsDistinct bool `protobuf:\"varint,7,opt,name=is_distinct,json=isDistinct,proto3\" json:\"is_distinct,omitempty\"`\n}\n\nfunc (x *CommonInlineUserDefinedFunction) Reset() {\n\t*x = CommonInlineUserDefinedFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CommonInlineUserDefinedFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CommonInlineUserDefinedFunction) ProtoMessage() {}\n\nfunc (x *CommonInlineUserDefinedFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CommonInlineUserDefinedFunction.ProtoReflect.Descriptor instead.\nfunc (*CommonInlineUserDefinedFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetDeterministic() bool {\n\tif x != nil {\n\t\treturn x.Deterministic\n\t}\n\treturn false\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\nfunc (m *CommonInlineUserDefinedFunction) GetFunction() isCommonInlineUserDefinedFunction_Function {\n\tif m != nil {\n\t\treturn m.Function\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetPythonUdf() *PythonUDF {\n\tif x, ok := x.GetFunction().(*CommonInlineUserDefinedFunction_PythonUdf); ok {\n\t\treturn x.PythonUdf\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetScalarScalaUdf() *ScalarScalaUDF {\n\tif x, ok := x.GetFunction().(*CommonInlineUserDefinedFunction_ScalarScalaUdf); ok {\n\t\treturn x.ScalarScalaUdf\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetJavaUdf() *JavaUDF {\n\tif x, ok := x.GetFunction().(*CommonInlineUserDefinedFunction_JavaUdf); ok {\n\t\treturn x.JavaUdf\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedFunction) GetIsDistinct() bool {\n\tif x != nil {\n\t\treturn x.IsDistinct\n\t}\n\treturn false\n}\n\ntype isCommonInlineUserDefinedFunction_Function interface {\n\tisCommonInlineUserDefinedFunction_Function()\n}\n\ntype CommonInlineUserDefinedFunction_PythonUdf struct {\n\tPythonUdf *PythonUDF `protobuf:\"bytes,4,opt,name=python_udf,json=pythonUdf,proto3,oneof\"`\n}\n\ntype CommonInlineUserDefinedFunction_ScalarScalaUdf struct {\n\tScalarScalaUdf *ScalarScalaUDF `protobuf:\"bytes,5,opt,name=scalar_scala_udf,json=scalarScalaUdf,proto3,oneof\"`\n}\n\ntype CommonInlineUserDefinedFunction_JavaUdf struct {\n\tJavaUdf *JavaUDF `protobuf:\"bytes,6,opt,name=java_udf,json=javaUdf,proto3,oneof\"`\n}\n\nfunc (*CommonInlineUserDefinedFunction_PythonUdf) isCommonInlineUserDefinedFunction_Function() {}\n\nfunc (*CommonInlineUserDefinedFunction_ScalarScalaUdf) isCommonInlineUserDefinedFunction_Function() {}\n\nfunc (*CommonInlineUserDefinedFunction_JavaUdf) isCommonInlineUserDefinedFunction_Function() {}\n\ntype PythonUDF struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Output type of the Python UDF\n\tOutputType *DataType `protobuf:\"bytes,1,opt,name=output_type,json=outputType,proto3\" json:\"output_type,omitempty\"`\n\t// (Required) EvalType of the Python UDF\n\tEvalType int32 `protobuf:\"varint,2,opt,name=eval_type,json=evalType,proto3\" json:\"eval_type,omitempty\"`\n\t// (Required) The encoded commands of the Python UDF\n\tCommand []byte `protobuf:\"bytes,3,opt,name=command,proto3\" json:\"command,omitempty\"`\n\t// (Required) Python version being used in the client.\n\tPythonVer string `protobuf:\"bytes,4,opt,name=python_ver,json=pythonVer,proto3\" json:\"python_ver,omitempty\"`\n\t// (Optional) Additional includes for the Python UDF.\n\tAdditionalIncludes []string `protobuf:\"bytes,5,rep,name=additional_includes,json=additionalIncludes,proto3\" json:\"additional_includes,omitempty\"`\n}\n\nfunc (x *PythonUDF) Reset() {\n\t*x = PythonUDF{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PythonUDF) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PythonUDF) ProtoMessage() {}\n\nfunc (x *PythonUDF) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PythonUDF.ProtoReflect.Descriptor instead.\nfunc (*PythonUDF) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *PythonUDF) GetOutputType() *DataType {\n\tif x != nil {\n\t\treturn x.OutputType\n\t}\n\treturn nil\n}\n\nfunc (x *PythonUDF) GetEvalType() int32 {\n\tif x != nil {\n\t\treturn x.EvalType\n\t}\n\treturn 0\n}\n\nfunc (x *PythonUDF) GetCommand() []byte {\n\tif x != nil {\n\t\treturn x.Command\n\t}\n\treturn nil\n}\n\nfunc (x *PythonUDF) GetPythonVer() string {\n\tif x != nil {\n\t\treturn x.PythonVer\n\t}\n\treturn \"\"\n}\n\nfunc (x *PythonUDF) GetAdditionalIncludes() []string {\n\tif x != nil {\n\t\treturn x.AdditionalIncludes\n\t}\n\treturn nil\n}\n\ntype ScalarScalaUDF struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Serialized JVM object containing UDF definition, input encoders and output encoder\n\tPayload []byte `protobuf:\"bytes,1,opt,name=payload,proto3\" json:\"payload,omitempty\"`\n\t// (Optional) Input type(s) of the UDF\n\tInputTypes []*DataType `protobuf:\"bytes,2,rep,name=inputTypes,proto3\" json:\"inputTypes,omitempty\"`\n\t// (Required) Output type of the UDF\n\tOutputType *DataType `protobuf:\"bytes,3,opt,name=outputType,proto3\" json:\"outputType,omitempty\"`\n\t// (Required) True if the UDF can return null value\n\tNullable bool `protobuf:\"varint,4,opt,name=nullable,proto3\" json:\"nullable,omitempty\"`\n\t// (Required) Indicate if the UDF is an aggregate function\n\tAggregate bool `protobuf:\"varint,5,opt,name=aggregate,proto3\" json:\"aggregate,omitempty\"`\n}\n\nfunc (x *ScalarScalaUDF) Reset() {\n\t*x = ScalarScalaUDF{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ScalarScalaUDF) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ScalarScalaUDF) ProtoMessage() {}\n\nfunc (x *ScalarScalaUDF) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ScalarScalaUDF.ProtoReflect.Descriptor instead.\nfunc (*ScalarScalaUDF) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *ScalarScalaUDF) GetPayload() []byte {\n\tif x != nil {\n\t\treturn x.Payload\n\t}\n\treturn nil\n}\n\nfunc (x *ScalarScalaUDF) GetInputTypes() []*DataType {\n\tif x != nil {\n\t\treturn x.InputTypes\n\t}\n\treturn nil\n}\n\nfunc (x *ScalarScalaUDF) GetOutputType() *DataType {\n\tif x != nil {\n\t\treturn x.OutputType\n\t}\n\treturn nil\n}\n\nfunc (x *ScalarScalaUDF) GetNullable() bool {\n\tif x != nil {\n\t\treturn x.Nullable\n\t}\n\treturn false\n}\n\nfunc (x *ScalarScalaUDF) GetAggregate() bool {\n\tif x != nil {\n\t\treturn x.Aggregate\n\t}\n\treturn false\n}\n\ntype JavaUDF struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Fully qualified name of Java class\n\tClassName string `protobuf:\"bytes,1,opt,name=class_name,json=className,proto3\" json:\"class_name,omitempty\"`\n\t// (Optional) Output type of the Java UDF\n\tOutputType *DataType `protobuf:\"bytes,2,opt,name=output_type,json=outputType,proto3,oneof\" json:\"output_type,omitempty\"`\n\t// (Required) Indicate if the Java user-defined function is an aggregate function\n\tAggregate bool `protobuf:\"varint,3,opt,name=aggregate,proto3\" json:\"aggregate,omitempty\"`\n}\n\nfunc (x *JavaUDF) Reset() {\n\t*x = JavaUDF{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *JavaUDF) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*JavaUDF) ProtoMessage() {}\n\nfunc (x *JavaUDF) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use JavaUDF.ProtoReflect.Descriptor instead.\nfunc (*JavaUDF) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (x *JavaUDF) GetClassName() string {\n\tif x != nil {\n\t\treturn x.ClassName\n\t}\n\treturn \"\"\n}\n\nfunc (x *JavaUDF) GetOutputType() *DataType {\n\tif x != nil {\n\t\treturn x.OutputType\n\t}\n\treturn nil\n}\n\nfunc (x *JavaUDF) GetAggregate() bool {\n\tif x != nil {\n\t\treturn x.Aggregate\n\t}\n\treturn false\n}\n\ntype TypedAggregateExpression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The aggregate function object packed into bytes.\n\tScalarScalaUdf *ScalarScalaUDF `protobuf:\"bytes,1,opt,name=scalar_scala_udf,json=scalarScalaUdf,proto3\" json:\"scalar_scala_udf,omitempty\"`\n}\n\nfunc (x *TypedAggregateExpression) Reset() {\n\t*x = TypedAggregateExpression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *TypedAggregateExpression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*TypedAggregateExpression) ProtoMessage() {}\n\nfunc (x *TypedAggregateExpression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use TypedAggregateExpression.ProtoReflect.Descriptor instead.\nfunc (*TypedAggregateExpression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (x *TypedAggregateExpression) GetScalarScalaUdf() *ScalarScalaUDF {\n\tif x != nil {\n\t\treturn x.ScalarScalaUdf\n\t}\n\treturn nil\n}\n\ntype CallFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Unparsed name of the SQL function.\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional) Function arguments. Empty arguments are allowed.\n\tArguments []*Expression `protobuf:\"bytes,2,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n}\n\nfunc (x *CallFunction) Reset() {\n\t*x = CallFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CallFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CallFunction) ProtoMessage() {}\n\nfunc (x *CallFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CallFunction.ProtoReflect.Descriptor instead.\nfunc (*CallFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (x *CallFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CallFunction) GetArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\ntype NamedArgumentExpression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The key of the named argument.\n\tKey string `protobuf:\"bytes,1,opt,name=key,proto3\" json:\"key,omitempty\"`\n\t// (Required) The value expression of the named argument.\n\tValue *Expression `protobuf:\"bytes,2,opt,name=value,proto3\" json:\"value,omitempty\"`\n}\n\nfunc (x *NamedArgumentExpression) Reset() {\n\t*x = NamedArgumentExpression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *NamedArgumentExpression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*NamedArgumentExpression) ProtoMessage() {}\n\nfunc (x *NamedArgumentExpression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use NamedArgumentExpression.ProtoReflect.Descriptor instead.\nfunc (*NamedArgumentExpression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *NamedArgumentExpression) GetKey() string {\n\tif x != nil {\n\t\treturn x.Key\n\t}\n\treturn \"\"\n}\n\nfunc (x *NamedArgumentExpression) GetValue() *Expression {\n\tif x != nil {\n\t\treturn x.Value\n\t}\n\treturn nil\n}\n\ntype MergeAction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The action type of the merge action.\n\tActionType MergeAction_ActionType `protobuf:\"varint,1,opt,name=action_type,json=actionType,proto3,enum=spark.connect.MergeAction_ActionType\" json:\"action_type,omitempty\"`\n\t// (Optional) The condition expression of the merge action.\n\tCondition *Expression `protobuf:\"bytes,2,opt,name=condition,proto3,oneof\" json:\"condition,omitempty\"`\n\t// (Optional) The assignments of the merge action. Required for ActionTypes INSERT and UPDATE.\n\tAssignments []*MergeAction_Assignment `protobuf:\"bytes,3,rep,name=assignments,proto3\" json:\"assignments,omitempty\"`\n}\n\nfunc (x *MergeAction) Reset() {\n\t*x = MergeAction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MergeAction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MergeAction) ProtoMessage() {}\n\nfunc (x *MergeAction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MergeAction.ProtoReflect.Descriptor instead.\nfunc (*MergeAction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *MergeAction) GetActionType() MergeAction_ActionType {\n\tif x != nil {\n\t\treturn x.ActionType\n\t}\n\treturn MergeAction_ACTION_TYPE_INVALID\n}\n\nfunc (x *MergeAction) GetCondition() *Expression {\n\tif x != nil {\n\t\treturn x.Condition\n\t}\n\treturn nil\n}\n\nfunc (x *MergeAction) GetAssignments() []*MergeAction_Assignment {\n\tif x != nil {\n\t\treturn x.Assignments\n\t}\n\treturn nil\n}\n\ntype SubqueryExpression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The ID of the corresponding connect plan.\n\tPlanId int64 `protobuf:\"varint,1,opt,name=plan_id,json=planId,proto3\" json:\"plan_id,omitempty\"`\n\t// (Required) The type of the subquery.\n\tSubqueryType SubqueryExpression_SubqueryType `protobuf:\"varint,2,opt,name=subquery_type,json=subqueryType,proto3,enum=spark.connect.SubqueryExpression_SubqueryType\" json:\"subquery_type,omitempty\"`\n\t// (Optional) Options specific to table arguments.\n\tTableArgOptions *SubqueryExpression_TableArgOptions `protobuf:\"bytes,3,opt,name=table_arg_options,json=tableArgOptions,proto3,oneof\" json:\"table_arg_options,omitempty\"`\n\t// (Optional) IN subquery values.\n\tInSubqueryValues []*Expression `protobuf:\"bytes,4,rep,name=in_subquery_values,json=inSubqueryValues,proto3\" json:\"in_subquery_values,omitempty\"`\n}\n\nfunc (x *SubqueryExpression) Reset() {\n\t*x = SubqueryExpression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SubqueryExpression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SubqueryExpression) ProtoMessage() {}\n\nfunc (x *SubqueryExpression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SubqueryExpression.ProtoReflect.Descriptor instead.\nfunc (*SubqueryExpression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *SubqueryExpression) GetPlanId() int64 {\n\tif x != nil {\n\t\treturn x.PlanId\n\t}\n\treturn 0\n}\n\nfunc (x *SubqueryExpression) GetSubqueryType() SubqueryExpression_SubqueryType {\n\tif x != nil {\n\t\treturn x.SubqueryType\n\t}\n\treturn SubqueryExpression_SUBQUERY_TYPE_UNKNOWN\n}\n\nfunc (x *SubqueryExpression) GetTableArgOptions() *SubqueryExpression_TableArgOptions {\n\tif x != nil {\n\t\treturn x.TableArgOptions\n\t}\n\treturn nil\n}\n\nfunc (x *SubqueryExpression) GetInSubqueryValues() []*Expression {\n\tif x != nil {\n\t\treturn x.InSubqueryValues\n\t}\n\treturn nil\n}\n\n// Expression for the OVER clause or WINDOW clause.\ntype Expression_Window struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The window function.\n\tWindowFunction *Expression `protobuf:\"bytes,1,opt,name=window_function,json=windowFunction,proto3\" json:\"window_function,omitempty\"`\n\t// (Optional) The way that input rows are partitioned.\n\tPartitionSpec []*Expression `protobuf:\"bytes,2,rep,name=partition_spec,json=partitionSpec,proto3\" json:\"partition_spec,omitempty\"`\n\t// (Optional) Ordering of rows in a partition.\n\tOrderSpec []*Expression_SortOrder `protobuf:\"bytes,3,rep,name=order_spec,json=orderSpec,proto3\" json:\"order_spec,omitempty\"`\n\t// (Optional) Window frame in a partition.\n\t//\n\t// If not set, it will be treated as 'UnspecifiedFrame'.\n\tFrameSpec *Expression_Window_WindowFrame `protobuf:\"bytes,4,opt,name=frame_spec,json=frameSpec,proto3\" json:\"frame_spec,omitempty\"`\n}\n\nfunc (x *Expression_Window) Reset() {\n\t*x = Expression_Window{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Window) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Window) ProtoMessage() {}\n\nfunc (x *Expression_Window) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Window.ProtoReflect.Descriptor instead.\nfunc (*Expression_Window) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 0}\n}\n\nfunc (x *Expression_Window) GetWindowFunction() *Expression {\n\tif x != nil {\n\t\treturn x.WindowFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Window) GetPartitionSpec() []*Expression {\n\tif x != nil {\n\t\treturn x.PartitionSpec\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Window) GetOrderSpec() []*Expression_SortOrder {\n\tif x != nil {\n\t\treturn x.OrderSpec\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Window) GetFrameSpec() *Expression_Window_WindowFrame {\n\tif x != nil {\n\t\treturn x.FrameSpec\n\t}\n\treturn nil\n}\n\n// SortOrder is used to specify the  data ordering, it is normally used in Sort and Window.\n// It is an unevaluable expression and cannot be evaluated, so can not be used in Projection.\ntype Expression_SortOrder struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The expression to be sorted.\n\tChild *Expression `protobuf:\"bytes,1,opt,name=child,proto3\" json:\"child,omitempty\"`\n\t// (Required) The sort direction, should be ASCENDING or DESCENDING.\n\tDirection Expression_SortOrder_SortDirection `protobuf:\"varint,2,opt,name=direction,proto3,enum=spark.connect.Expression_SortOrder_SortDirection\" json:\"direction,omitempty\"`\n\t// (Required) How to deal with NULLs, should be NULLS_FIRST or NULLS_LAST.\n\tNullOrdering Expression_SortOrder_NullOrdering `protobuf:\"varint,3,opt,name=null_ordering,json=nullOrdering,proto3,enum=spark.connect.Expression_SortOrder_NullOrdering\" json:\"null_ordering,omitempty\"`\n}\n\nfunc (x *Expression_SortOrder) Reset() {\n\t*x = Expression_SortOrder{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_SortOrder) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_SortOrder) ProtoMessage() {}\n\nfunc (x *Expression_SortOrder) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_SortOrder.ProtoReflect.Descriptor instead.\nfunc (*Expression_SortOrder) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 1}\n}\n\nfunc (x *Expression_SortOrder) GetChild() *Expression {\n\tif x != nil {\n\t\treturn x.Child\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_SortOrder) GetDirection() Expression_SortOrder_SortDirection {\n\tif x != nil {\n\t\treturn x.Direction\n\t}\n\treturn Expression_SortOrder_SORT_DIRECTION_UNSPECIFIED\n}\n\nfunc (x *Expression_SortOrder) GetNullOrdering() Expression_SortOrder_NullOrdering {\n\tif x != nil {\n\t\treturn x.NullOrdering\n\t}\n\treturn Expression_SortOrder_SORT_NULLS_UNSPECIFIED\n}\n\ntype Expression_Cast struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) the expression to be casted.\n\tExpr *Expression `protobuf:\"bytes,1,opt,name=expr,proto3\" json:\"expr,omitempty\"`\n\t// (Required) the data type that the expr to be casted to.\n\t//\n\t// Types that are assignable to CastToType:\n\t//\n\t//\t*Expression_Cast_Type\n\t//\t*Expression_Cast_TypeStr\n\tCastToType isExpression_Cast_CastToType `protobuf_oneof:\"cast_to_type\"`\n\t// (Optional) The expression evaluation mode.\n\tEvalMode Expression_Cast_EvalMode `protobuf:\"varint,4,opt,name=eval_mode,json=evalMode,proto3,enum=spark.connect.Expression_Cast_EvalMode\" json:\"eval_mode,omitempty\"`\n}\n\nfunc (x *Expression_Cast) Reset() {\n\t*x = Expression_Cast{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Cast) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Cast) ProtoMessage() {}\n\nfunc (x *Expression_Cast) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Cast.ProtoReflect.Descriptor instead.\nfunc (*Expression_Cast) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 2}\n}\n\nfunc (x *Expression_Cast) GetExpr() *Expression {\n\tif x != nil {\n\t\treturn x.Expr\n\t}\n\treturn nil\n}\n\nfunc (m *Expression_Cast) GetCastToType() isExpression_Cast_CastToType {\n\tif m != nil {\n\t\treturn m.CastToType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Cast) GetType() *DataType {\n\tif x, ok := x.GetCastToType().(*Expression_Cast_Type); ok {\n\t\treturn x.Type\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Cast) GetTypeStr() string {\n\tif x, ok := x.GetCastToType().(*Expression_Cast_TypeStr); ok {\n\t\treturn x.TypeStr\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_Cast) GetEvalMode() Expression_Cast_EvalMode {\n\tif x != nil {\n\t\treturn x.EvalMode\n\t}\n\treturn Expression_Cast_EVAL_MODE_UNSPECIFIED\n}\n\ntype isExpression_Cast_CastToType interface {\n\tisExpression_Cast_CastToType()\n}\n\ntype Expression_Cast_Type struct {\n\tType *DataType `protobuf:\"bytes,2,opt,name=type,proto3,oneof\"`\n}\n\ntype Expression_Cast_TypeStr struct {\n\t// If this is set, Server will use Catalyst parser to parse this string to DataType.\n\tTypeStr string `protobuf:\"bytes,3,opt,name=type_str,json=typeStr,proto3,oneof\"`\n}\n\nfunc (*Expression_Cast_Type) isExpression_Cast_CastToType() {}\n\nfunc (*Expression_Cast_TypeStr) isExpression_Cast_CastToType() {}\n\ntype Expression_Literal struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to LiteralType:\n\t//\n\t//\t*Expression_Literal_Null\n\t//\t*Expression_Literal_Binary\n\t//\t*Expression_Literal_Boolean\n\t//\t*Expression_Literal_Byte\n\t//\t*Expression_Literal_Short\n\t//\t*Expression_Literal_Integer\n\t//\t*Expression_Literal_Long\n\t//\t*Expression_Literal_Float\n\t//\t*Expression_Literal_Double\n\t//\t*Expression_Literal_Decimal_\n\t//\t*Expression_Literal_String_\n\t//\t*Expression_Literal_Date\n\t//\t*Expression_Literal_Timestamp\n\t//\t*Expression_Literal_TimestampNtz\n\t//\t*Expression_Literal_CalendarInterval_\n\t//\t*Expression_Literal_YearMonthInterval\n\t//\t*Expression_Literal_DayTimeInterval\n\t//\t*Expression_Literal_Array_\n\t//\t*Expression_Literal_Map_\n\t//\t*Expression_Literal_Struct_\n\t//\t*Expression_Literal_SpecializedArray_\n\tLiteralType isExpression_Literal_LiteralType `protobuf_oneof:\"literal_type\"`\n}\n\nfunc (x *Expression_Literal) Reset() {\n\t*x = Expression_Literal{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal) ProtoMessage() {}\n\nfunc (x *Expression_Literal) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3}\n}\n\nfunc (m *Expression_Literal) GetLiteralType() isExpression_Literal_LiteralType {\n\tif m != nil {\n\t\treturn m.LiteralType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetNull() *DataType {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Null); ok {\n\t\treturn x.Null\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetBinary() []byte {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Binary); ok {\n\t\treturn x.Binary\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetBoolean() bool {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Boolean); ok {\n\t\treturn x.Boolean\n\t}\n\treturn false\n}\n\nfunc (x *Expression_Literal) GetByte() int32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Byte); ok {\n\t\treturn x.Byte\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetShort() int32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Short); ok {\n\t\treturn x.Short\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetInteger() int32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Integer); ok {\n\t\treturn x.Integer\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetLong() int64 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Long); ok {\n\t\treturn x.Long\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetFloat() float32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Float); ok {\n\t\treturn x.Float\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetDouble() float64 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Double); ok {\n\t\treturn x.Double\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetDecimal() *Expression_Literal_Decimal {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Decimal_); ok {\n\t\treturn x.Decimal\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetString_() string {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_String_); ok {\n\t\treturn x.String_\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_Literal) GetDate() int32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Date); ok {\n\t\treturn x.Date\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetTimestamp() int64 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Timestamp); ok {\n\t\treturn x.Timestamp\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetTimestampNtz() int64 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_TimestampNtz); ok {\n\t\treturn x.TimestampNtz\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetCalendarInterval() *Expression_Literal_CalendarInterval {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_CalendarInterval_); ok {\n\t\treturn x.CalendarInterval\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetYearMonthInterval() int32 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_YearMonthInterval); ok {\n\t\treturn x.YearMonthInterval\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetDayTimeInterval() int64 {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_DayTimeInterval); ok {\n\t\treturn x.DayTimeInterval\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal) GetArray() *Expression_Literal_Array {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Array_); ok {\n\t\treturn x.Array\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetMap() *Expression_Literal_Map {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Map_); ok {\n\t\treturn x.Map\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetStruct() *Expression_Literal_Struct {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_Struct_); ok {\n\t\treturn x.Struct\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal) GetSpecializedArray() *Expression_Literal_SpecializedArray {\n\tif x, ok := x.GetLiteralType().(*Expression_Literal_SpecializedArray_); ok {\n\t\treturn x.SpecializedArray\n\t}\n\treturn nil\n}\n\ntype isExpression_Literal_LiteralType interface {\n\tisExpression_Literal_LiteralType()\n}\n\ntype Expression_Literal_Null struct {\n\tNull *DataType `protobuf:\"bytes,1,opt,name=null,proto3,oneof\"`\n}\n\ntype Expression_Literal_Binary struct {\n\tBinary []byte `protobuf:\"bytes,2,opt,name=binary,proto3,oneof\"`\n}\n\ntype Expression_Literal_Boolean struct {\n\tBoolean bool `protobuf:\"varint,3,opt,name=boolean,proto3,oneof\"`\n}\n\ntype Expression_Literal_Byte struct {\n\tByte int32 `protobuf:\"varint,4,opt,name=byte,proto3,oneof\"`\n}\n\ntype Expression_Literal_Short struct {\n\tShort int32 `protobuf:\"varint,5,opt,name=short,proto3,oneof\"`\n}\n\ntype Expression_Literal_Integer struct {\n\tInteger int32 `protobuf:\"varint,6,opt,name=integer,proto3,oneof\"`\n}\n\ntype Expression_Literal_Long struct {\n\tLong int64 `protobuf:\"varint,7,opt,name=long,proto3,oneof\"`\n}\n\ntype Expression_Literal_Float struct {\n\tFloat float32 `protobuf:\"fixed32,10,opt,name=float,proto3,oneof\"`\n}\n\ntype Expression_Literal_Double struct {\n\tDouble float64 `protobuf:\"fixed64,11,opt,name=double,proto3,oneof\"`\n}\n\ntype Expression_Literal_Decimal_ struct {\n\tDecimal *Expression_Literal_Decimal `protobuf:\"bytes,12,opt,name=decimal,proto3,oneof\"`\n}\n\ntype Expression_Literal_String_ struct {\n\tString_ string `protobuf:\"bytes,13,opt,name=string,proto3,oneof\"`\n}\n\ntype Expression_Literal_Date struct {\n\t// Date in units of days since the UNIX epoch.\n\tDate int32 `protobuf:\"varint,16,opt,name=date,proto3,oneof\"`\n}\n\ntype Expression_Literal_Timestamp struct {\n\t// Timestamp in units of microseconds since the UNIX epoch.\n\tTimestamp int64 `protobuf:\"varint,17,opt,name=timestamp,proto3,oneof\"`\n}\n\ntype Expression_Literal_TimestampNtz struct {\n\t// Timestamp in units of microseconds since the UNIX epoch (without timezone information).\n\tTimestampNtz int64 `protobuf:\"varint,18,opt,name=timestamp_ntz,json=timestampNtz,proto3,oneof\"`\n}\n\ntype Expression_Literal_CalendarInterval_ struct {\n\tCalendarInterval *Expression_Literal_CalendarInterval `protobuf:\"bytes,19,opt,name=calendar_interval,json=calendarInterval,proto3,oneof\"`\n}\n\ntype Expression_Literal_YearMonthInterval struct {\n\tYearMonthInterval int32 `protobuf:\"varint,20,opt,name=year_month_interval,json=yearMonthInterval,proto3,oneof\"`\n}\n\ntype Expression_Literal_DayTimeInterval struct {\n\tDayTimeInterval int64 `protobuf:\"varint,21,opt,name=day_time_interval,json=dayTimeInterval,proto3,oneof\"`\n}\n\ntype Expression_Literal_Array_ struct {\n\tArray *Expression_Literal_Array `protobuf:\"bytes,22,opt,name=array,proto3,oneof\"`\n}\n\ntype Expression_Literal_Map_ struct {\n\tMap *Expression_Literal_Map `protobuf:\"bytes,23,opt,name=map,proto3,oneof\"`\n}\n\ntype Expression_Literal_Struct_ struct {\n\tStruct *Expression_Literal_Struct `protobuf:\"bytes,24,opt,name=struct,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_ struct {\n\tSpecializedArray *Expression_Literal_SpecializedArray `protobuf:\"bytes,25,opt,name=specialized_array,json=specializedArray,proto3,oneof\"`\n}\n\nfunc (*Expression_Literal_Null) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Binary) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Boolean) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Byte) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Short) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Integer) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Long) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Float) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Double) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Decimal_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_String_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Date) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Timestamp) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_TimestampNtz) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_CalendarInterval_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_YearMonthInterval) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_DayTimeInterval) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Array_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Map_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_Struct_) isExpression_Literal_LiteralType() {}\n\nfunc (*Expression_Literal_SpecializedArray_) isExpression_Literal_LiteralType() {}\n\n// An unresolved attribute that is not explicitly bound to a specific column, but the column\n// is resolved during analysis by name.\ntype Expression_UnresolvedAttribute struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) An identifier that will be parsed by Catalyst parser. This should follow the\n\t// Spark SQL identifier syntax.\n\tUnparsedIdentifier string `protobuf:\"bytes,1,opt,name=unparsed_identifier,json=unparsedIdentifier,proto3\" json:\"unparsed_identifier,omitempty\"`\n\t// (Optional) The id of corresponding connect plan.\n\tPlanId *int64 `protobuf:\"varint,2,opt,name=plan_id,json=planId,proto3,oneof\" json:\"plan_id,omitempty\"`\n\t// (Optional) The requested column is a metadata column.\n\tIsMetadataColumn *bool `protobuf:\"varint,3,opt,name=is_metadata_column,json=isMetadataColumn,proto3,oneof\" json:\"is_metadata_column,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedAttribute) Reset() {\n\t*x = Expression_UnresolvedAttribute{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedAttribute) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedAttribute) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedAttribute) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedAttribute.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedAttribute) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 4}\n}\n\nfunc (x *Expression_UnresolvedAttribute) GetUnparsedIdentifier() string {\n\tif x != nil {\n\t\treturn x.UnparsedIdentifier\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_UnresolvedAttribute) GetPlanId() int64 {\n\tif x != nil && x.PlanId != nil {\n\t\treturn *x.PlanId\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_UnresolvedAttribute) GetIsMetadataColumn() bool {\n\tif x != nil && x.IsMetadataColumn != nil {\n\t\treturn *x.IsMetadataColumn\n\t}\n\treturn false\n}\n\n// An unresolved function is not explicitly bound to one explicit function, but the function\n// is resolved during analysis following Sparks name resolution rules.\ntype Expression_UnresolvedFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) name (or unparsed name for user defined function) for the unresolved function.\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional) Function arguments. Empty arguments are allowed.\n\tArguments []*Expression `protobuf:\"bytes,2,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n\t// (Required) Indicate if this function should be applied on distinct values.\n\tIsDistinct bool `protobuf:\"varint,3,opt,name=is_distinct,json=isDistinct,proto3\" json:\"is_distinct,omitempty\"`\n\t// (Required) Indicate if this is a user defined function.\n\t//\n\t// When it is not a user defined function, Connect will use the function name directly.\n\t// When it is a user defined function, Connect will parse the function name first.\n\tIsUserDefinedFunction bool `protobuf:\"varint,4,opt,name=is_user_defined_function,json=isUserDefinedFunction,proto3\" json:\"is_user_defined_function,omitempty\"`\n\t// (Optional) Indicate if this function is defined in the internal function registry.\n\t// If not set, the server will try to look up the function in the internal function registry\n\t// and decide appropriately.\n\tIsInternal *bool `protobuf:\"varint,5,opt,name=is_internal,json=isInternal,proto3,oneof\" json:\"is_internal,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedFunction) Reset() {\n\t*x = Expression_UnresolvedFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedFunction) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedFunction.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 5}\n}\n\nfunc (x *Expression_UnresolvedFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_UnresolvedFunction) GetArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_UnresolvedFunction) GetIsDistinct() bool {\n\tif x != nil {\n\t\treturn x.IsDistinct\n\t}\n\treturn false\n}\n\nfunc (x *Expression_UnresolvedFunction) GetIsUserDefinedFunction() bool {\n\tif x != nil {\n\t\treturn x.IsUserDefinedFunction\n\t}\n\treturn false\n}\n\nfunc (x *Expression_UnresolvedFunction) GetIsInternal() bool {\n\tif x != nil && x.IsInternal != nil {\n\t\treturn *x.IsInternal\n\t}\n\treturn false\n}\n\n// Expression as string.\ntype Expression_ExpressionString struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) A SQL expression that will be parsed by Catalyst parser.\n\tExpression string `protobuf:\"bytes,1,opt,name=expression,proto3\" json:\"expression,omitempty\"`\n}\n\nfunc (x *Expression_ExpressionString) Reset() {\n\t*x = Expression_ExpressionString{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_ExpressionString) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_ExpressionString) ProtoMessage() {}\n\nfunc (x *Expression_ExpressionString) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_ExpressionString.ProtoReflect.Descriptor instead.\nfunc (*Expression_ExpressionString) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 6}\n}\n\nfunc (x *Expression_ExpressionString) GetExpression() string {\n\tif x != nil {\n\t\treturn x.Expression\n\t}\n\treturn \"\"\n}\n\n// UnresolvedStar is used to expand all the fields of a relation or struct.\ntype Expression_UnresolvedStar struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The target of the expansion.\n\t//\n\t// If set, it should end with '.*' and will be parsed by 'parseAttributeName'\n\t// in the server side.\n\tUnparsedTarget *string `protobuf:\"bytes,1,opt,name=unparsed_target,json=unparsedTarget,proto3,oneof\" json:\"unparsed_target,omitempty\"`\n\t// (Optional) The id of corresponding connect plan.\n\tPlanId *int64 `protobuf:\"varint,2,opt,name=plan_id,json=planId,proto3,oneof\" json:\"plan_id,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedStar) Reset() {\n\t*x = Expression_UnresolvedStar{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedStar) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedStar) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedStar) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedStar.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedStar) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 7}\n}\n\nfunc (x *Expression_UnresolvedStar) GetUnparsedTarget() string {\n\tif x != nil && x.UnparsedTarget != nil {\n\t\treturn *x.UnparsedTarget\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_UnresolvedStar) GetPlanId() int64 {\n\tif x != nil && x.PlanId != nil {\n\t\treturn *x.PlanId\n\t}\n\treturn 0\n}\n\n// Represents all of the input attributes to a given relational operator, for example in\n// \"SELECT `(id)?+.+` FROM ...\".\ntype Expression_UnresolvedRegex struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The column name used to extract column with regex.\n\tColName string `protobuf:\"bytes,1,opt,name=col_name,json=colName,proto3\" json:\"col_name,omitempty\"`\n\t// (Optional) The id of corresponding connect plan.\n\tPlanId *int64 `protobuf:\"varint,2,opt,name=plan_id,json=planId,proto3,oneof\" json:\"plan_id,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedRegex) Reset() {\n\t*x = Expression_UnresolvedRegex{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedRegex) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedRegex) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedRegex) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedRegex.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedRegex) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 8}\n}\n\nfunc (x *Expression_UnresolvedRegex) GetColName() string {\n\tif x != nil {\n\t\treturn x.ColName\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_UnresolvedRegex) GetPlanId() int64 {\n\tif x != nil && x.PlanId != nil {\n\t\treturn *x.PlanId\n\t}\n\treturn 0\n}\n\n// Extracts a value or values from an Expression\ntype Expression_UnresolvedExtractValue struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The expression to extract value from, can be\n\t// Map, Array, Struct or array of Structs.\n\tChild *Expression `protobuf:\"bytes,1,opt,name=child,proto3\" json:\"child,omitempty\"`\n\t// (Required) The expression to describe the extraction, can be\n\t// key of Map, index of Array, field name of Struct.\n\tExtraction *Expression `protobuf:\"bytes,2,opt,name=extraction,proto3\" json:\"extraction,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedExtractValue) Reset() {\n\t*x = Expression_UnresolvedExtractValue{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedExtractValue) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedExtractValue) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedExtractValue) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedExtractValue.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedExtractValue) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 9}\n}\n\nfunc (x *Expression_UnresolvedExtractValue) GetChild() *Expression {\n\tif x != nil {\n\t\treturn x.Child\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_UnresolvedExtractValue) GetExtraction() *Expression {\n\tif x != nil {\n\t\treturn x.Extraction\n\t}\n\treturn nil\n}\n\n// Add, replace or drop a field of `StructType` expression by name.\ntype Expression_UpdateFields struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The struct expression.\n\tStructExpression *Expression `protobuf:\"bytes,1,opt,name=struct_expression,json=structExpression,proto3\" json:\"struct_expression,omitempty\"`\n\t// (Required) The field name.\n\tFieldName string `protobuf:\"bytes,2,opt,name=field_name,json=fieldName,proto3\" json:\"field_name,omitempty\"`\n\t// (Optional) The expression to add or replace.\n\t//\n\t// When not set, it means this field will be dropped.\n\tValueExpression *Expression `protobuf:\"bytes,3,opt,name=value_expression,json=valueExpression,proto3\" json:\"value_expression,omitempty\"`\n}\n\nfunc (x *Expression_UpdateFields) Reset() {\n\t*x = Expression_UpdateFields{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UpdateFields) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UpdateFields) ProtoMessage() {}\n\nfunc (x *Expression_UpdateFields) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UpdateFields.ProtoReflect.Descriptor instead.\nfunc (*Expression_UpdateFields) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 10}\n}\n\nfunc (x *Expression_UpdateFields) GetStructExpression() *Expression {\n\tif x != nil {\n\t\treturn x.StructExpression\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_UpdateFields) GetFieldName() string {\n\tif x != nil {\n\t\treturn x.FieldName\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_UpdateFields) GetValueExpression() *Expression {\n\tif x != nil {\n\t\treturn x.ValueExpression\n\t}\n\treturn nil\n}\n\ntype Expression_Alias struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The expression that alias will be added on.\n\tExpr *Expression `protobuf:\"bytes,1,opt,name=expr,proto3\" json:\"expr,omitempty\"`\n\t// (Required) a list of name parts for the alias.\n\t//\n\t// Scalar columns only has one name that presents.\n\tName []string `protobuf:\"bytes,2,rep,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Optional) Alias metadata expressed as a JSON map.\n\tMetadata *string `protobuf:\"bytes,3,opt,name=metadata,proto3,oneof\" json:\"metadata,omitempty\"`\n}\n\nfunc (x *Expression_Alias) Reset() {\n\t*x = Expression_Alias{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Alias) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Alias) ProtoMessage() {}\n\nfunc (x *Expression_Alias) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Alias.ProtoReflect.Descriptor instead.\nfunc (*Expression_Alias) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 11}\n}\n\nfunc (x *Expression_Alias) GetExpr() *Expression {\n\tif x != nil {\n\t\treturn x.Expr\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Alias) GetName() []string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Alias) GetMetadata() string {\n\tif x != nil && x.Metadata != nil {\n\t\treturn *x.Metadata\n\t}\n\treturn \"\"\n}\n\ntype Expression_LambdaFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The lambda function.\n\t//\n\t// The function body should use 'UnresolvedAttribute' as arguments, the sever side will\n\t// replace 'UnresolvedAttribute' with 'UnresolvedNamedLambdaVariable'.\n\tFunction *Expression `protobuf:\"bytes,1,opt,name=function,proto3\" json:\"function,omitempty\"`\n\t// (Required) Function variables. Must contains 1 ~ 3 variables.\n\tArguments []*Expression_UnresolvedNamedLambdaVariable `protobuf:\"bytes,2,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n}\n\nfunc (x *Expression_LambdaFunction) Reset() {\n\t*x = Expression_LambdaFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_LambdaFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_LambdaFunction) ProtoMessage() {}\n\nfunc (x *Expression_LambdaFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_LambdaFunction.ProtoReflect.Descriptor instead.\nfunc (*Expression_LambdaFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 12}\n}\n\nfunc (x *Expression_LambdaFunction) GetFunction() *Expression {\n\tif x != nil {\n\t\treturn x.Function\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_LambdaFunction) GetArguments() []*Expression_UnresolvedNamedLambdaVariable {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\ntype Expression_UnresolvedNamedLambdaVariable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) a list of name parts for the variable. Must not be empty.\n\tNameParts []string `protobuf:\"bytes,1,rep,name=name_parts,json=nameParts,proto3\" json:\"name_parts,omitempty\"`\n}\n\nfunc (x *Expression_UnresolvedNamedLambdaVariable) Reset() {\n\t*x = Expression_UnresolvedNamedLambdaVariable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[24]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_UnresolvedNamedLambdaVariable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_UnresolvedNamedLambdaVariable) ProtoMessage() {}\n\nfunc (x *Expression_UnresolvedNamedLambdaVariable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[24]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_UnresolvedNamedLambdaVariable.ProtoReflect.Descriptor instead.\nfunc (*Expression_UnresolvedNamedLambdaVariable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 13}\n}\n\nfunc (x *Expression_UnresolvedNamedLambdaVariable) GetNameParts() []string {\n\tif x != nil {\n\t\treturn x.NameParts\n\t}\n\treturn nil\n}\n\n// The window frame\ntype Expression_Window_WindowFrame struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The type of the frame.\n\tFrameType Expression_Window_WindowFrame_FrameType `protobuf:\"varint,1,opt,name=frame_type,json=frameType,proto3,enum=spark.connect.Expression_Window_WindowFrame_FrameType\" json:\"frame_type,omitempty\"`\n\t// (Required) The lower bound of the frame.\n\tLower *Expression_Window_WindowFrame_FrameBoundary `protobuf:\"bytes,2,opt,name=lower,proto3\" json:\"lower,omitempty\"`\n\t// (Required) The upper bound of the frame.\n\tUpper *Expression_Window_WindowFrame_FrameBoundary `protobuf:\"bytes,3,opt,name=upper,proto3\" json:\"upper,omitempty\"`\n}\n\nfunc (x *Expression_Window_WindowFrame) Reset() {\n\t*x = Expression_Window_WindowFrame{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[25]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Window_WindowFrame) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Window_WindowFrame) ProtoMessage() {}\n\nfunc (x *Expression_Window_WindowFrame) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[25]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Window_WindowFrame.ProtoReflect.Descriptor instead.\nfunc (*Expression_Window_WindowFrame) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 0, 0}\n}\n\nfunc (x *Expression_Window_WindowFrame) GetFrameType() Expression_Window_WindowFrame_FrameType {\n\tif x != nil {\n\t\treturn x.FrameType\n\t}\n\treturn Expression_Window_WindowFrame_FRAME_TYPE_UNDEFINED\n}\n\nfunc (x *Expression_Window_WindowFrame) GetLower() *Expression_Window_WindowFrame_FrameBoundary {\n\tif x != nil {\n\t\treturn x.Lower\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Window_WindowFrame) GetUpper() *Expression_Window_WindowFrame_FrameBoundary {\n\tif x != nil {\n\t\treturn x.Upper\n\t}\n\treturn nil\n}\n\ntype Expression_Window_WindowFrame_FrameBoundary struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Boundary:\n\t//\n\t//\t*Expression_Window_WindowFrame_FrameBoundary_CurrentRow\n\t//\t*Expression_Window_WindowFrame_FrameBoundary_Unbounded\n\t//\t*Expression_Window_WindowFrame_FrameBoundary_Value\n\tBoundary isExpression_Window_WindowFrame_FrameBoundary_Boundary `protobuf_oneof:\"boundary\"`\n}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) Reset() {\n\t*x = Expression_Window_WindowFrame_FrameBoundary{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[26]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Window_WindowFrame_FrameBoundary) ProtoMessage() {}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[26]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Window_WindowFrame_FrameBoundary.ProtoReflect.Descriptor instead.\nfunc (*Expression_Window_WindowFrame_FrameBoundary) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 0, 0, 0}\n}\n\nfunc (m *Expression_Window_WindowFrame_FrameBoundary) GetBoundary() isExpression_Window_WindowFrame_FrameBoundary_Boundary {\n\tif m != nil {\n\t\treturn m.Boundary\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) GetCurrentRow() bool {\n\tif x, ok := x.GetBoundary().(*Expression_Window_WindowFrame_FrameBoundary_CurrentRow); ok {\n\t\treturn x.CurrentRow\n\t}\n\treturn false\n}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) GetUnbounded() bool {\n\tif x, ok := x.GetBoundary().(*Expression_Window_WindowFrame_FrameBoundary_Unbounded); ok {\n\t\treturn x.Unbounded\n\t}\n\treturn false\n}\n\nfunc (x *Expression_Window_WindowFrame_FrameBoundary) GetValue() *Expression {\n\tif x, ok := x.GetBoundary().(*Expression_Window_WindowFrame_FrameBoundary_Value); ok {\n\t\treturn x.Value\n\t}\n\treturn nil\n}\n\ntype isExpression_Window_WindowFrame_FrameBoundary_Boundary interface {\n\tisExpression_Window_WindowFrame_FrameBoundary_Boundary()\n}\n\ntype Expression_Window_WindowFrame_FrameBoundary_CurrentRow struct {\n\t// CURRENT ROW boundary\n\tCurrentRow bool `protobuf:\"varint,1,opt,name=current_row,json=currentRow,proto3,oneof\"`\n}\n\ntype Expression_Window_WindowFrame_FrameBoundary_Unbounded struct {\n\t// UNBOUNDED boundary.\n\t// For lower bound, it will be converted to 'UnboundedPreceding'.\n\t// for upper bound, it will be converted to 'UnboundedFollowing'.\n\tUnbounded bool `protobuf:\"varint,2,opt,name=unbounded,proto3,oneof\"`\n}\n\ntype Expression_Window_WindowFrame_FrameBoundary_Value struct {\n\t// This is an expression for future proofing. We are expecting literals on the server side.\n\tValue *Expression `protobuf:\"bytes,3,opt,name=value,proto3,oneof\"`\n}\n\nfunc (*Expression_Window_WindowFrame_FrameBoundary_CurrentRow) isExpression_Window_WindowFrame_FrameBoundary_Boundary() {\n}\n\nfunc (*Expression_Window_WindowFrame_FrameBoundary_Unbounded) isExpression_Window_WindowFrame_FrameBoundary_Boundary() {\n}\n\nfunc (*Expression_Window_WindowFrame_FrameBoundary_Value) isExpression_Window_WindowFrame_FrameBoundary_Boundary() {\n}\n\ntype Expression_Literal_Decimal struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// the string representation.\n\tValue string `protobuf:\"bytes,1,opt,name=value,proto3\" json:\"value,omitempty\"`\n\t// The maximum number of digits allowed in the value.\n\t// the maximum precision is 38.\n\tPrecision *int32 `protobuf:\"varint,2,opt,name=precision,proto3,oneof\" json:\"precision,omitempty\"`\n\t// declared scale of decimal literal\n\tScale *int32 `protobuf:\"varint,3,opt,name=scale,proto3,oneof\" json:\"scale,omitempty\"`\n}\n\nfunc (x *Expression_Literal_Decimal) Reset() {\n\t*x = Expression_Literal_Decimal{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[27]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_Decimal) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_Decimal) ProtoMessage() {}\n\nfunc (x *Expression_Literal_Decimal) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[27]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_Decimal.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_Decimal) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 0}\n}\n\nfunc (x *Expression_Literal_Decimal) GetValue() string {\n\tif x != nil {\n\t\treturn x.Value\n\t}\n\treturn \"\"\n}\n\nfunc (x *Expression_Literal_Decimal) GetPrecision() int32 {\n\tif x != nil && x.Precision != nil {\n\t\treturn *x.Precision\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal_Decimal) GetScale() int32 {\n\tif x != nil && x.Scale != nil {\n\t\treturn *x.Scale\n\t}\n\treturn 0\n}\n\ntype Expression_Literal_CalendarInterval struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tMonths       int32 `protobuf:\"varint,1,opt,name=months,proto3\" json:\"months,omitempty\"`\n\tDays         int32 `protobuf:\"varint,2,opt,name=days,proto3\" json:\"days,omitempty\"`\n\tMicroseconds int64 `protobuf:\"varint,3,opt,name=microseconds,proto3\" json:\"microseconds,omitempty\"`\n}\n\nfunc (x *Expression_Literal_CalendarInterval) Reset() {\n\t*x = Expression_Literal_CalendarInterval{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[28]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_CalendarInterval) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_CalendarInterval) ProtoMessage() {}\n\nfunc (x *Expression_Literal_CalendarInterval) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[28]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_CalendarInterval.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_CalendarInterval) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 1}\n}\n\nfunc (x *Expression_Literal_CalendarInterval) GetMonths() int32 {\n\tif x != nil {\n\t\treturn x.Months\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal_CalendarInterval) GetDays() int32 {\n\tif x != nil {\n\t\treturn x.Days\n\t}\n\treturn 0\n}\n\nfunc (x *Expression_Literal_CalendarInterval) GetMicroseconds() int64 {\n\tif x != nil {\n\t\treturn x.Microseconds\n\t}\n\treturn 0\n}\n\ntype Expression_Literal_Array struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tElementType *DataType             `protobuf:\"bytes,1,opt,name=element_type,json=elementType,proto3\" json:\"element_type,omitempty\"`\n\tElements    []*Expression_Literal `protobuf:\"bytes,2,rep,name=elements,proto3\" json:\"elements,omitempty\"`\n}\n\nfunc (x *Expression_Literal_Array) Reset() {\n\t*x = Expression_Literal_Array{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[29]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_Array) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_Array) ProtoMessage() {}\n\nfunc (x *Expression_Literal_Array) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[29]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_Array.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_Array) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 2}\n}\n\nfunc (x *Expression_Literal_Array) GetElementType() *DataType {\n\tif x != nil {\n\t\treturn x.ElementType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_Array) GetElements() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Elements\n\t}\n\treturn nil\n}\n\ntype Expression_Literal_Map struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tKeyType   *DataType             `protobuf:\"bytes,1,opt,name=key_type,json=keyType,proto3\" json:\"key_type,omitempty\"`\n\tValueType *DataType             `protobuf:\"bytes,2,opt,name=value_type,json=valueType,proto3\" json:\"value_type,omitempty\"`\n\tKeys      []*Expression_Literal `protobuf:\"bytes,3,rep,name=keys,proto3\" json:\"keys,omitempty\"`\n\tValues    []*Expression_Literal `protobuf:\"bytes,4,rep,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Expression_Literal_Map) Reset() {\n\t*x = Expression_Literal_Map{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[30]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_Map) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_Map) ProtoMessage() {}\n\nfunc (x *Expression_Literal_Map) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[30]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_Map.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_Map) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 3}\n}\n\nfunc (x *Expression_Literal_Map) GetKeyType() *DataType {\n\tif x != nil {\n\t\treturn x.KeyType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_Map) GetValueType() *DataType {\n\tif x != nil {\n\t\treturn x.ValueType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_Map) GetKeys() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Keys\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_Map) GetValues() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Expression_Literal_Struct struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tStructType *DataType             `protobuf:\"bytes,1,opt,name=struct_type,json=structType,proto3\" json:\"struct_type,omitempty\"`\n\tElements   []*Expression_Literal `protobuf:\"bytes,2,rep,name=elements,proto3\" json:\"elements,omitempty\"`\n}\n\nfunc (x *Expression_Literal_Struct) Reset() {\n\t*x = Expression_Literal_Struct{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[31]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_Struct) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_Struct) ProtoMessage() {}\n\nfunc (x *Expression_Literal_Struct) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[31]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_Struct.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_Struct) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 4}\n}\n\nfunc (x *Expression_Literal_Struct) GetStructType() *DataType {\n\tif x != nil {\n\t\treturn x.StructType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_Struct) GetElements() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Elements\n\t}\n\treturn nil\n}\n\ntype Expression_Literal_SpecializedArray struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ValueType:\n\t//\n\t//\t*Expression_Literal_SpecializedArray_Bools\n\t//\t*Expression_Literal_SpecializedArray_Ints\n\t//\t*Expression_Literal_SpecializedArray_Longs\n\t//\t*Expression_Literal_SpecializedArray_Floats\n\t//\t*Expression_Literal_SpecializedArray_Doubles\n\t//\t*Expression_Literal_SpecializedArray_Strings\n\tValueType isExpression_Literal_SpecializedArray_ValueType `protobuf_oneof:\"value_type\"`\n}\n\nfunc (x *Expression_Literal_SpecializedArray) Reset() {\n\t*x = Expression_Literal_SpecializedArray{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[32]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Expression_Literal_SpecializedArray) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Expression_Literal_SpecializedArray) ProtoMessage() {}\n\nfunc (x *Expression_Literal_SpecializedArray) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[32]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Expression_Literal_SpecializedArray.ProtoReflect.Descriptor instead.\nfunc (*Expression_Literal_SpecializedArray) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{0, 3, 5}\n}\n\nfunc (m *Expression_Literal_SpecializedArray) GetValueType() isExpression_Literal_SpecializedArray_ValueType {\n\tif m != nil {\n\t\treturn m.ValueType\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetBools() *Bools {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Bools); ok {\n\t\treturn x.Bools\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetInts() *Ints {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Ints); ok {\n\t\treturn x.Ints\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetLongs() *Longs {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Longs); ok {\n\t\treturn x.Longs\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetFloats() *Floats {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Floats); ok {\n\t\treturn x.Floats\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetDoubles() *Doubles {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Doubles); ok {\n\t\treturn x.Doubles\n\t}\n\treturn nil\n}\n\nfunc (x *Expression_Literal_SpecializedArray) GetStrings() *Strings {\n\tif x, ok := x.GetValueType().(*Expression_Literal_SpecializedArray_Strings); ok {\n\t\treturn x.Strings\n\t}\n\treturn nil\n}\n\ntype isExpression_Literal_SpecializedArray_ValueType interface {\n\tisExpression_Literal_SpecializedArray_ValueType()\n}\n\ntype Expression_Literal_SpecializedArray_Bools struct {\n\tBools *Bools `protobuf:\"bytes,1,opt,name=bools,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_Ints struct {\n\tInts *Ints `protobuf:\"bytes,2,opt,name=ints,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_Longs struct {\n\tLongs *Longs `protobuf:\"bytes,3,opt,name=longs,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_Floats struct {\n\tFloats *Floats `protobuf:\"bytes,4,opt,name=floats,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_Doubles struct {\n\tDoubles *Doubles `protobuf:\"bytes,5,opt,name=doubles,proto3,oneof\"`\n}\n\ntype Expression_Literal_SpecializedArray_Strings struct {\n\tStrings *Strings `protobuf:\"bytes,6,opt,name=strings,proto3,oneof\"`\n}\n\nfunc (*Expression_Literal_SpecializedArray_Bools) isExpression_Literal_SpecializedArray_ValueType() {}\n\nfunc (*Expression_Literal_SpecializedArray_Ints) isExpression_Literal_SpecializedArray_ValueType() {}\n\nfunc (*Expression_Literal_SpecializedArray_Longs) isExpression_Literal_SpecializedArray_ValueType() {}\n\nfunc (*Expression_Literal_SpecializedArray_Floats) isExpression_Literal_SpecializedArray_ValueType() {\n}\n\nfunc (*Expression_Literal_SpecializedArray_Doubles) isExpression_Literal_SpecializedArray_ValueType() {\n}\n\nfunc (*Expression_Literal_SpecializedArray_Strings) isExpression_Literal_SpecializedArray_ValueType() {\n}\n\ntype MergeAction_Assignment struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The key of the assignment.\n\tKey *Expression `protobuf:\"bytes,1,opt,name=key,proto3\" json:\"key,omitempty\"`\n\t// (Required) The value of the assignment.\n\tValue *Expression `protobuf:\"bytes,2,opt,name=value,proto3\" json:\"value,omitempty\"`\n}\n\nfunc (x *MergeAction_Assignment) Reset() {\n\t*x = MergeAction_Assignment{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[33]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MergeAction_Assignment) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MergeAction_Assignment) ProtoMessage() {}\n\nfunc (x *MergeAction_Assignment) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[33]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MergeAction_Assignment.ProtoReflect.Descriptor instead.\nfunc (*MergeAction_Assignment) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{9, 0}\n}\n\nfunc (x *MergeAction_Assignment) GetKey() *Expression {\n\tif x != nil {\n\t\treturn x.Key\n\t}\n\treturn nil\n}\n\nfunc (x *MergeAction_Assignment) GetValue() *Expression {\n\tif x != nil {\n\t\treturn x.Value\n\t}\n\treturn nil\n}\n\n// Nested message for table argument options.\ntype SubqueryExpression_TableArgOptions struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) The way that input rows are partitioned.\n\tPartitionSpec []*Expression `protobuf:\"bytes,1,rep,name=partition_spec,json=partitionSpec,proto3\" json:\"partition_spec,omitempty\"`\n\t// (Optional) Ordering of rows in a partition.\n\tOrderSpec []*Expression_SortOrder `protobuf:\"bytes,2,rep,name=order_spec,json=orderSpec,proto3\" json:\"order_spec,omitempty\"`\n\t// (Optional) Whether this is a single partition.\n\tWithSinglePartition *bool `protobuf:\"varint,3,opt,name=with_single_partition,json=withSinglePartition,proto3,oneof\" json:\"with_single_partition,omitempty\"`\n}\n\nfunc (x *SubqueryExpression_TableArgOptions) Reset() {\n\t*x = SubqueryExpression_TableArgOptions{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_expressions_proto_msgTypes[34]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SubqueryExpression_TableArgOptions) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SubqueryExpression_TableArgOptions) ProtoMessage() {}\n\nfunc (x *SubqueryExpression_TableArgOptions) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_expressions_proto_msgTypes[34]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SubqueryExpression_TableArgOptions.ProtoReflect.Descriptor instead.\nfunc (*SubqueryExpression_TableArgOptions) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_expressions_proto_rawDescGZIP(), []int{10, 0}\n}\n\nfunc (x *SubqueryExpression_TableArgOptions) GetPartitionSpec() []*Expression {\n\tif x != nil {\n\t\treturn x.PartitionSpec\n\t}\n\treturn nil\n}\n\nfunc (x *SubqueryExpression_TableArgOptions) GetOrderSpec() []*Expression_SortOrder {\n\tif x != nil {\n\t\treturn x.OrderSpec\n\t}\n\treturn nil\n}\n\nfunc (x *SubqueryExpression_TableArgOptions) GetWithSinglePartition() bool {\n\tif x != nil && x.WithSinglePartition != nil {\n\t\treturn *x.WithSinglePartition\n\t}\n\treturn false\n}\n\nvar File_spark_connect_expressions_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_expressions_proto_rawDesc = []byte{\n\t0x0a, 0x1f, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,\n\t0x6f, 0x12, 0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,\n\t0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73,\n\t0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1a, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x22, 0xf3, 0x34, 0x0a, 0x0a, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x12, 0x37, 0x0a, 0x06, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x18, 0x12, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x6f, 0x6e, 0x52, 0x06, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x12, 0x3d, 0x0a, 0x07, 0x6c, 0x69,\n\t0x74, 0x65, 0x72, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x48, 0x00,\n\t0x52, 0x07, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x12, 0x62, 0x0a, 0x14, 0x75, 0x6e, 0x72,\n\t0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74,\n\t0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x41, 0x74, 0x74,\n\t0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x48, 0x00, 0x52, 0x13, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f,\n\t0x6c, 0x76, 0x65, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x5f, 0x0a,\n\t0x13, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x66, 0x75, 0x6e, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64,\n\t0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x12, 0x75, 0x6e, 0x72, 0x65,\n\t0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x59,\n\t0x0a, 0x11, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x72,\n\t0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x53,\n\t0x74, 0x72, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x10, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x53, 0x0a, 0x0f, 0x75, 0x6e, 0x72,\n\t0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x18, 0x05, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x6e,\n\t0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x53, 0x74, 0x61, 0x72, 0x48, 0x00, 0x52, 0x0e,\n\t0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x53, 0x74, 0x61, 0x72, 0x12, 0x37,\n\t0x0a, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x48, 0x00,\n\t0x52, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x34, 0x0a, 0x04, 0x63, 0x61, 0x73, 0x74, 0x18,\n\t0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x2e, 0x43, 0x61, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x63, 0x61, 0x73, 0x74, 0x12, 0x56, 0x0a,\n\t0x10, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x67, 0x65,\n\t0x78, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x52, 0x65, 0x67,\n\t0x65, 0x78, 0x48, 0x00, 0x52, 0x0f, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64,\n\t0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x44, 0x0a, 0x0a, 0x73, 0x6f, 0x72, 0x74, 0x5f, 0x6f, 0x72,\n\t0x64, 0x65, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x48, 0x00,\n\t0x52, 0x09, 0x73, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x53, 0x0a, 0x0f, 0x6c,\n\t0x61, 0x6d, 0x62, 0x64, 0x61, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0a,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x4c, 0x61, 0x6d, 0x62, 0x64, 0x61, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00,\n\t0x52, 0x0e, 0x6c, 0x61, 0x6d, 0x62, 0x64, 0x61, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x12, 0x3a, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x20, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x57, 0x69, 0x6e, 0x64,\n\t0x6f, 0x77, 0x48, 0x00, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x6c, 0x0a, 0x18,\n\t0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61,\n\t0x63, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f,\n\t0x6c, 0x76, 0x65, 0x64, 0x45, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65,\n\t0x48, 0x00, 0x52, 0x16, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x45, 0x78,\n\t0x74, 0x72, 0x61, 0x63, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x4d, 0x0a, 0x0d, 0x75, 0x70,\n\t0x64, 0x61, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64,\n\t0x61, 0x74, 0x65, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x48, 0x00, 0x52, 0x0c, 0x75, 0x70, 0x64,\n\t0x61, 0x74, 0x65, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x82, 0x01, 0x0a, 0x20, 0x75, 0x6e,\n\t0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x5f, 0x6c,\n\t0x61, 0x6d, 0x62, 0x64, 0x61, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0e,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x4c,\n\t0x61, 0x6d, 0x62, 0x64, 0x61, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52,\n\t0x1d, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x64,\n\t0x4c, 0x61, 0x6d, 0x62, 0x64, 0x61, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x7e,\n\t0x0a, 0x23, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x5f,\n\t0x75, 0x73, 0x65, 0x72, 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x5f, 0x66, 0x75, 0x6e,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69,\n\t0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x1f, 0x63,\n\t0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44,\n\t0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x42,\n\t0x0a, 0x0d, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18,\n\t0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x61, 0x6c, 0x6c, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0c, 0x63, 0x61, 0x6c, 0x6c, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x12, 0x64, 0x0a, 0x19, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x5f, 0x61, 0x72, 0x67, 0x75,\n\t0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18,\n\t0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67, 0x75, 0x6d,\n\t0x65, 0x6e, 0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52,\n\t0x17, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x0c, 0x6d, 0x65, 0x72, 0x67,\n\t0x65, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d,\n\t0x65, 0x72, 0x67, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x6d, 0x65,\n\t0x72, 0x67, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x67, 0x0a, 0x1a, 0x74, 0x79, 0x70,\n\t0x65, 0x64, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x65, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x79,\n\t0x70, 0x65, 0x64, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x18, 0x74, 0x79, 0x70, 0x65, 0x64, 0x41,\n\t0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x12, 0x54, 0x0a, 0x13, 0x73, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x65,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x48, 0x00, 0x52, 0x12, 0x73, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65,\n\t0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67,\n\t0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41,\n\t0x6e, 0x79, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x1a,\n\t0x8f, 0x06, 0x0a, 0x06, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x42, 0x0a, 0x0f, 0x77, 0x69,\n\t0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0e,\n\t0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40,\n\t0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x70, 0x65, 0x63,\n\t0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63,\n\t0x12, 0x42, 0x0a, 0x0a, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x52, 0x09, 0x6f, 0x72, 0x64, 0x65, 0x72,\n\t0x53, 0x70, 0x65, 0x63, 0x12, 0x4b, 0x0a, 0x0a, 0x66, 0x72, 0x61, 0x6d, 0x65, 0x5f, 0x73, 0x70,\n\t0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f,\n\t0x77, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x52, 0x09, 0x66, 0x72, 0x61, 0x6d, 0x65, 0x53, 0x70, 0x65,\n\t0x63, 0x1a, 0xed, 0x03, 0x0a, 0x0b, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x46, 0x72, 0x61, 0x6d,\n\t0x65, 0x12, 0x55, 0x0a, 0x0a, 0x66, 0x72, 0x61, 0x6d, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x46, 0x72,\n\t0x61, 0x6d, 0x65, 0x2e, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x66,\n\t0x72, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x50, 0x0a, 0x05, 0x6c, 0x6f, 0x77, 0x65,\n\t0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77,\n\t0x46, 0x72, 0x61, 0x6d, 0x65, 0x2e, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x42, 0x6f, 0x75, 0x6e, 0x64,\n\t0x61, 0x72, 0x79, 0x52, 0x05, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x12, 0x50, 0x0a, 0x05, 0x75, 0x70,\n\t0x70, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x2e, 0x57, 0x69, 0x6e, 0x64,\n\t0x6f, 0x77, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x2e, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x42, 0x6f, 0x75,\n\t0x6e, 0x64, 0x61, 0x72, 0x79, 0x52, 0x05, 0x75, 0x70, 0x70, 0x65, 0x72, 0x1a, 0x91, 0x01, 0x0a,\n\t0x0d, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x12, 0x21,\n\t0x0a, 0x0b, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x6f,\n\t0x77, 0x12, 0x1e, 0x0a, 0x09, 0x75, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x75, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65,\n\t0x64, 0x12, 0x31, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x61, 0x72, 0x79,\n\t0x22, 0x4f, 0x0a, 0x09, 0x46, 0x72, 0x61, 0x6d, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a,\n\t0x14, 0x46, 0x52, 0x41, 0x4d, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x44, 0x45,\n\t0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x46, 0x52, 0x41, 0x4d, 0x45,\n\t0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x46,\n\t0x52, 0x41, 0x4d, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x41, 0x4e, 0x47, 0x45, 0x10,\n\t0x02, 0x1a, 0xa9, 0x03, 0x0a, 0x09, 0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x12,\n\t0x2f, 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64,\n\t0x12, 0x4f, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x53,\n\t0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72,\n\t0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x12, 0x55, 0x0a, 0x0d, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x69,\n\t0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x2e, 0x4e, 0x75,\n\t0x6c, 0x6c, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x0c, 0x6e, 0x75, 0x6c, 0x6c,\n\t0x4f, 0x72, 0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x22, 0x6c, 0x0a, 0x0d, 0x53, 0x6f, 0x72, 0x74,\n\t0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x4f, 0x52,\n\t0x54, 0x5f, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50,\n\t0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1c, 0x0a, 0x18, 0x53, 0x4f, 0x52,\n\t0x54, 0x5f, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x53, 0x43, 0x45,\n\t0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x4f, 0x52, 0x54, 0x5f,\n\t0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x45, 0x53, 0x43, 0x45, 0x4e,\n\t0x44, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x22, 0x55, 0x0a, 0x0c, 0x4e, 0x75, 0x6c, 0x6c, 0x4f, 0x72,\n\t0x64, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x4f, 0x52, 0x54, 0x5f, 0x4e,\n\t0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44,\n\t0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x4f, 0x52, 0x54, 0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53,\n\t0x5f, 0x46, 0x49, 0x52, 0x53, 0x54, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x4f, 0x52, 0x54,\n\t0x5f, 0x4e, 0x55, 0x4c, 0x4c, 0x53, 0x5f, 0x4c, 0x41, 0x53, 0x54, 0x10, 0x02, 0x1a, 0xbb, 0x02,\n\t0x0a, 0x04, 0x43, 0x61, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x04, 0x65, 0x78, 0x70, 0x72, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52,\n\t0x04, 0x65, 0x78, 0x70, 0x72, 0x12, 0x2d, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x04,\n\t0x74, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x08, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x73, 0x74, 0x72,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x74, 0x79, 0x70, 0x65, 0x53, 0x74,\n\t0x72, 0x12, 0x44, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x04,\n\t0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x43, 0x61, 0x73, 0x74, 0x2e, 0x45, 0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x08, 0x65,\n\t0x76, 0x61, 0x6c, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x62, 0x0a, 0x08, 0x45, 0x76, 0x61, 0x6c, 0x4d,\n\t0x6f, 0x64, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x45, 0x56, 0x41, 0x4c, 0x5f, 0x4d, 0x4f, 0x44, 0x45,\n\t0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x14,\n\t0x0a, 0x10, 0x45, 0x56, 0x41, 0x4c, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4c, 0x45, 0x47, 0x41,\n\t0x43, 0x59, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x45, 0x56, 0x41, 0x4c, 0x5f, 0x4d, 0x4f, 0x44,\n\t0x45, 0x5f, 0x41, 0x4e, 0x53, 0x49, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, 0x45, 0x56, 0x41, 0x4c,\n\t0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x54, 0x52, 0x59, 0x10, 0x03, 0x42, 0x0e, 0x0a, 0x0c, 0x63,\n\t0x61, 0x73, 0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x1a, 0xc1, 0x0f, 0x0a, 0x07,\n\t0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x12, 0x2d, 0x0a, 0x04, 0x6e, 0x75, 0x6c, 0x6c, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00,\n\t0x52, 0x04, 0x6e, 0x75, 0x6c, 0x6c, 0x12, 0x18, 0x0a, 0x06, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x06, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79,\n\t0x12, 0x1a, 0x0a, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x65, 0x61, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x08, 0x48, 0x00, 0x52, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x65, 0x61, 0x6e, 0x12, 0x14, 0x0a, 0x04,\n\t0x62, 0x79, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x04, 0x62, 0x79,\n\t0x74, 0x65, 0x12, 0x16, 0x0a, 0x05, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28,\n\t0x05, 0x48, 0x00, 0x52, 0x05, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x07, 0x69, 0x6e,\n\t0x74, 0x65, 0x67, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x07, 0x69,\n\t0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x04, 0x6c, 0x6f, 0x6e, 0x67, 0x18, 0x07,\n\t0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x04, 0x6c, 0x6f, 0x6e, 0x67, 0x12, 0x16, 0x0a, 0x05,\n\t0x66, 0x6c, 0x6f, 0x61, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, 0x05, 0x66,\n\t0x6c, 0x6f, 0x61, 0x74, 0x12, 0x18, 0x0a, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x18, 0x0b,\n\t0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x12, 0x45,\n\t0x0a, 0x07, 0x64, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72,\n\t0x61, 0x6c, 0x2e, 0x44, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x07, 0x64, 0x65,\n\t0x63, 0x69, 0x6d, 0x61, 0x6c, 0x12, 0x18, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18,\n\t0x0d, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12,\n\t0x14, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52,\n\t0x04, 0x64, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,\n\t0x6d, 0x70, 0x18, 0x11, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65,\n\t0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,\n\t0x6d, 0x70, 0x5f, 0x6e, 0x74, 0x7a, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0c,\n\t0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x74, 0x7a, 0x12, 0x61, 0x0a, 0x11,\n\t0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61,\n\t0x6c, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x2e, 0x43, 0x61, 0x6c, 0x65, 0x6e,\n\t0x64, 0x61, 0x72, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x10, 0x63,\n\t0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12,\n\t0x30, 0x0a, 0x13, 0x79, 0x65, 0x61, 0x72, 0x5f, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x5f, 0x69, 0x6e,\n\t0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x14, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x11,\n\t0x79, 0x65, 0x61, 0x72, 0x4d, 0x6f, 0x6e, 0x74, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61,\n\t0x6c, 0x12, 0x2c, 0x0a, 0x11, 0x64, 0x61, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x69, 0x6e,\n\t0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0f,\n\t0x64, 0x61, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12,\n\t0x3f, 0x0a, 0x05, 0x61, 0x72, 0x72, 0x61, 0x79, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61,\n\t0x6c, 0x2e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x48, 0x00, 0x52, 0x05, 0x61, 0x72, 0x72, 0x61, 0x79,\n\t0x12, 0x39, 0x0a, 0x03, 0x6d, 0x61, 0x70, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x2e, 0x4d, 0x61, 0x70, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x61, 0x70, 0x12, 0x42, 0x0a, 0x06, 0x73,\n\t0x74, 0x72, 0x75, 0x63, 0x74, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x2e, 0x53,\n\t0x74, 0x72, 0x75, 0x63, 0x74, 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x12,\n\t0x61, 0x0a, 0x11, 0x73, 0x70, 0x65, 0x63, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x61,\n\t0x72, 0x72, 0x61, 0x79, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x2e, 0x53, 0x70,\n\t0x65, 0x63, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x41, 0x72, 0x72, 0x61, 0x79, 0x48, 0x00,\n\t0x52, 0x10, 0x73, 0x70, 0x65, 0x63, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x41, 0x72, 0x72,\n\t0x61, 0x79, 0x1a, 0x75, 0x0a, 0x07, 0x44, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x12, 0x14, 0x0a,\n\t0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73,\n\t0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x19, 0x0a, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18,\n\t0x03, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x88, 0x01,\n\t0x01, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x42,\n\t0x08, 0x0a, 0x06, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0x62, 0x0a, 0x10, 0x43, 0x61, 0x6c,\n\t0x65, 0x6e, 0x64, 0x61, 0x72, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x16, 0x0a,\n\t0x06, 0x6d, 0x6f, 0x6e, 0x74, 0x68, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x6d,\n\t0x6f, 0x6e, 0x74, 0x68, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x79, 0x73, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x05, 0x52, 0x04, 0x64, 0x61, 0x79, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x6d, 0x69, 0x63,\n\t0x72, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52,\n\t0x0c, 0x6d, 0x69, 0x63, 0x72, 0x6f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x1a, 0x82, 0x01,\n\t0x0a, 0x05, 0x41, 0x72, 0x72, 0x61, 0x79, 0x12, 0x3a, 0x0a, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65,\n\t0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61,\n\t0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54,\n\t0x79, 0x70, 0x65, 0x12, 0x3d, 0x0a, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18,\n\t0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,\n\t0x74, 0x73, 0x1a, 0xe3, 0x01, 0x0a, 0x03, 0x4d, 0x61, 0x70, 0x12, 0x32, 0x0a, 0x08, 0x6b, 0x65,\n\t0x79, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74,\n\t0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6b, 0x65, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x36,\n\t0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x35, 0x0a, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x04, 0x6b, 0x65, 0x79, 0x73, 0x12, 0x39, 0x0a,\n\t0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x1a, 0x81, 0x01, 0x0a, 0x06, 0x53, 0x74, 0x72,\n\t0x75, 0x63, 0x74, 0x12, 0x38, 0x0a, 0x0b, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x5f, 0x74, 0x79,\n\t0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70,\n\t0x65, 0x52, 0x0a, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x3d, 0x0a,\n\t0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72,\n\t0x61, 0x6c, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0xc0, 0x02, 0x0a,\n\t0x10, 0x53, 0x70, 0x65, 0x63, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x41, 0x72, 0x72, 0x61,\n\t0x79, 0x12, 0x2c, 0x0a, 0x05, 0x62, 0x6f, 0x6f, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x14, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x73, 0x48, 0x00, 0x52, 0x05, 0x62, 0x6f, 0x6f, 0x6c, 0x73, 0x12,\n\t0x29, 0x0a, 0x04, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x49, 0x6e,\n\t0x74, 0x73, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x0a, 0x05, 0x6c, 0x6f,\n\t0x6e, 0x67, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x6f, 0x6e, 0x67, 0x73, 0x48,\n\t0x00, 0x52, 0x05, 0x6c, 0x6f, 0x6e, 0x67, 0x73, 0x12, 0x2f, 0x0a, 0x06, 0x66, 0x6c, 0x6f, 0x61,\n\t0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x73, 0x48,\n\t0x00, 0x52, 0x06, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x73, 0x12, 0x32, 0x0a, 0x07, 0x64, 0x6f, 0x75,\n\t0x62, 0x6c, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x6f, 0x75, 0x62, 0x6c,\n\t0x65, 0x73, 0x48, 0x00, 0x52, 0x07, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x32, 0x0a,\n\t0x07, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53,\n\t0x74, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x48, 0x00, 0x52, 0x07, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67,\n\t0x73, 0x42, 0x0c, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x42,\n\t0x0e, 0x0a, 0x0c, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x1a,\n\t0xba, 0x01, 0x0a, 0x13, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x41, 0x74,\n\t0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x2f, 0x0a, 0x13, 0x75, 0x6e, 0x70, 0x61, 0x72,\n\t0x73, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x75, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x49, 0x64,\n\t0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e,\n\t0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x06, 0x70, 0x6c, 0x61,\n\t0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x31, 0x0a, 0x12, 0x69, 0x73, 0x5f, 0x6d, 0x65, 0x74,\n\t0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x08, 0x48, 0x01, 0x52, 0x10, 0x69, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61,\n\t0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x6c,\n\t0x61, 0x6e, 0x5f, 0x69, 0x64, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x69, 0x73, 0x5f, 0x6d, 0x65, 0x74,\n\t0x61, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x82, 0x02, 0x0a,\n\t0x12, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x75,\n\t0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74,\n\t0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e,\n\t0x63, 0x74, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x64, 0x65,\n\t0x66, 0x69, 0x6e, 0x65, 0x64, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04,\n\t0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x73, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69,\n\t0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0b, 0x69,\n\t0x73, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08,\n\t0x48, 0x00, 0x52, 0x0a, 0x69, 0x73, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x88, 0x01,\n\t0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x69, 0x73, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,\n\t0x6c, 0x1a, 0x32, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x53,\n\t0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1e, 0x0a, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x65, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0x7c, 0x0a, 0x0e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c,\n\t0x76, 0x65, 0x64, 0x53, 0x74, 0x61, 0x72, 0x12, 0x2c, 0x0a, 0x0f, 0x75, 0x6e, 0x70, 0x61, 0x72,\n\t0x73, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x0e, 0x75, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x54, 0x61, 0x72, 0x67,\n\t0x65, 0x74, 0x88, 0x01, 0x01, 0x12, 0x1c, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69, 0x64,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x01, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x64,\n\t0x88, 0x01, 0x01, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x75, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64,\n\t0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x6c, 0x61, 0x6e,\n\t0x5f, 0x69, 0x64, 0x1a, 0x56, 0x0a, 0x0f, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65,\n\t0x64, 0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x6c, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x4e, 0x61, 0x6d,\n\t0x65, 0x12, 0x1c, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x03, 0x48, 0x00, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x42,\n\t0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69, 0x64, 0x1a, 0x84, 0x01, 0x0a, 0x16,\n\t0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x45, 0x78, 0x74, 0x72, 0x61, 0x63,\n\t0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2f, 0x0a, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x52, 0x05, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x1a, 0xbb, 0x01, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x46, 0x69, 0x65,\n\t0x6c, 0x64, 0x73, 0x12, 0x46, 0x0a, 0x11, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x5f, 0x65, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x73, 0x74, 0x72, 0x75, 0x63,\n\t0x74, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x66,\n\t0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x44, 0x0a, 0x10, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52,\n\t0x0f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x1a, 0x78, 0x0a, 0x05, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x2d, 0x0a, 0x04, 0x65, 0x78, 0x70,\n\t0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x52, 0x04, 0x65, 0x78, 0x70, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x08,\n\t0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00,\n\t0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b, 0x0a,\n\t0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x9e, 0x01, 0x0a, 0x0e, 0x4c,\n\t0x61, 0x6d, 0x62, 0x64, 0x61, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a,\n\t0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x66, 0x75, 0x6e, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74,\n\t0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x4e, 0x61, 0x6d,\n\t0x65, 0x64, 0x4c, 0x61, 0x6d, 0x62, 0x64, 0x61, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65,\n\t0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x3e, 0x0a, 0x1d, 0x55,\n\t0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x4c, 0x61,\n\t0x6d, 0x62, 0x64, 0x61, 0x56, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a,\n\t0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09,\n\t0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x50, 0x61, 0x72, 0x74, 0x73, 0x42, 0x0b, 0x0a, 0x09, 0x65,\n\t0x78, 0x70, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x41, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x06,\n\t0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x72, 0x69,\n\t0x67, 0x69, 0x6e, 0x52, 0x06, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x22, 0x8d, 0x03, 0x0a, 0x1f,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72,\n\t0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12,\n\t0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x64, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e,\n\t0x69, 0x73, 0x74, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64, 0x65, 0x74,\n\t0x65, 0x72, 0x6d, 0x69, 0x6e, 0x69, 0x73, 0x74, 0x69, 0x63, 0x12, 0x37, 0x0a, 0x09, 0x61, 0x72,\n\t0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65,\n\t0x6e, 0x74, 0x73, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x75, 0x64,\n\t0x66, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x55, 0x44,\n\t0x46, 0x48, 0x00, 0x52, 0x09, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x55, 0x64, 0x66, 0x12, 0x49,\n\t0x0a, 0x10, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x5f, 0x75,\n\t0x64, 0x66, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x53,\n\t0x63, 0x61, 0x6c, 0x61, 0x55, 0x44, 0x46, 0x48, 0x00, 0x52, 0x0e, 0x73, 0x63, 0x61, 0x6c, 0x61,\n\t0x72, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x55, 0x64, 0x66, 0x12, 0x33, 0x0a, 0x08, 0x6a, 0x61, 0x76,\n\t0x61, 0x5f, 0x75, 0x64, 0x66, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4a, 0x61, 0x76, 0x61,\n\t0x55, 0x44, 0x46, 0x48, 0x00, 0x52, 0x07, 0x6a, 0x61, 0x76, 0x61, 0x55, 0x64, 0x66, 0x12, 0x1f,\n\t0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x18, 0x07, 0x20,\n\t0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x42,\n\t0x0a, 0x0a, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xcc, 0x01, 0x0a, 0x09,\n\t0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x55, 0x44, 0x46, 0x12, 0x38, 0x0a, 0x0b, 0x6f, 0x75, 0x74,\n\t0x70, 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54,\n\t0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65,\n\t0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x0c, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x79,\n\t0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x76, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09,\n\t0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x56, 0x65, 0x72, 0x12, 0x2f, 0x0a, 0x13, 0x61, 0x64, 0x64,\n\t0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73,\n\t0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,\n\t0x61, 0x6c, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x73, 0x22, 0xd6, 0x01, 0x0a, 0x0e, 0x53,\n\t0x63, 0x61, 0x6c, 0x61, 0x72, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x55, 0x44, 0x46, 0x12, 0x18, 0x0a,\n\t0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07,\n\t0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x37, 0x0a, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x54, 0x79, 0x70, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61,\n\t0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73,\n\t0x12, 0x37, 0x0a, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x6f,\n\t0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x75, 0x6c,\n\t0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c,\n\t0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,\n\t0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67,\n\t0x61, 0x74, 0x65, 0x22, 0x95, 0x01, 0x0a, 0x07, 0x4a, 0x61, 0x76, 0x61, 0x55, 0x44, 0x46, 0x12,\n\t0x1d, 0x0a, 0x0a, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x3d,\n\t0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a,\n\t0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x1c, 0x0a,\n\t0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08,\n\t0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x42, 0x0e, 0x0a, 0x0c, 0x5f,\n\t0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x63, 0x0a, 0x18, 0x54,\n\t0x79, 0x70, 0x65, 0x64, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x47, 0x0a, 0x10, 0x73, 0x63, 0x61, 0x6c, 0x61,\n\t0x72, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x5f, 0x75, 0x64, 0x66, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x55, 0x44, 0x46,\n\t0x52, 0x0e, 0x73, 0x63, 0x61, 0x6c, 0x61, 0x72, 0x53, 0x63, 0x61, 0x6c, 0x61, 0x55, 0x64, 0x66,\n\t0x22, 0x6c, 0x0a, 0x0c, 0x43, 0x61, 0x6c, 0x6c, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e,\n\t0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x5c,\n\t0x0a, 0x17, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2f, 0x0a, 0x05, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x80, 0x04, 0x0a,\n\t0x0b, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x0b,\n\t0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0e, 0x32, 0x25, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x54, 0x79, 0x70, 0x65, 0x12, 0x3c, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f,\n\t0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x48, 0x00, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x88,\n\t0x01, 0x01, 0x12, 0x47, 0x0a, 0x0b, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74,\n\t0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x41, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b,\n\t0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x6a, 0x0a, 0x0a, 0x41,\n\t0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x6b, 0x65, 0x79,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2f, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xa7, 0x01, 0x0a, 0x0a, 0x41, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x13, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e,\n\t0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x00, 0x12,\n\t0x16, 0x0a, 0x12, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44,\n\t0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x41, 0x43, 0x54, 0x49, 0x4f,\n\t0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x53, 0x45, 0x52, 0x54, 0x10, 0x02, 0x12,\n\t0x1b, 0x0a, 0x17, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49,\n\t0x4e, 0x53, 0x45, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x10, 0x03, 0x12, 0x16, 0x0a, 0x12,\n\t0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x44, 0x41,\n\t0x54, 0x45, 0x10, 0x04, 0x12, 0x1b, 0x0a, 0x17, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54,\n\t0x59, 0x50, 0x45, 0x5f, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x10,\n\t0x05, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22,\n\t0xc5, 0x05, 0x0a, 0x12, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69,\n\t0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x64, 0x12,\n\t0x53, 0x0a, 0x0d, 0x73, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x74, 0x79, 0x70, 0x65,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65,\n\t0x72, 0x79, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79,\n\t0x54, 0x79, 0x70, 0x65, 0x12, 0x62, 0x0a, 0x11, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x61, 0x72,\n\t0x67, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x31, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x72, 0x67, 0x4f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x73, 0x48, 0x00, 0x52, 0x0f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x72, 0x67, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x88, 0x01, 0x01, 0x12, 0x47, 0x0a, 0x12, 0x69, 0x6e, 0x5f, 0x73,\n\t0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x04,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52,\n\t0x10, 0x69, 0x6e, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65,\n\t0x73, 0x1a, 0xea, 0x01, 0x0a, 0x0f, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x41, 0x72, 0x67, 0x4f, 0x70,\n\t0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x40, 0x0a, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,\n\t0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x42, 0x0a, 0x0a, 0x6f, 0x72, 0x64, 0x65, 0x72,\n\t0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72,\n\t0x52, 0x09, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x53, 0x70, 0x65, 0x63, 0x12, 0x37, 0x0a, 0x15, 0x77,\n\t0x69, 0x74, 0x68, 0x5f, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69,\n\t0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x13, 0x77, 0x69,\n\t0x74, 0x68, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,\n\t0x6e, 0x88, 0x01, 0x01, 0x42, 0x18, 0x0a, 0x16, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x69,\n\t0x6e, 0x67, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x90,\n\t0x01, 0x0a, 0x0c, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12,\n\t0x19, 0x0a, 0x15, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45,\n\t0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x55,\n\t0x42, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x43, 0x41, 0x4c,\n\t0x41, 0x52, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x59,\n\t0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x58, 0x49, 0x53, 0x54, 0x53, 0x10, 0x02, 0x12, 0x1b,\n\t0x0a, 0x17, 0x53, 0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,\n\t0x54, 0x41, 0x42, 0x4c, 0x45, 0x5f, 0x41, 0x52, 0x47, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x53,\n\t0x55, 0x42, 0x51, 0x55, 0x45, 0x52, 0x59, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x10,\n\t0x04, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x61, 0x72, 0x67, 0x5f,\n\t0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61,\n\t0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74,\n\t0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62,\n\t0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_expressions_proto_rawDescOnce sync.Once\n\tfile_spark_connect_expressions_proto_rawDescData = file_spark_connect_expressions_proto_rawDesc\n)\n\nfunc file_spark_connect_expressions_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_expressions_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_expressions_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_expressions_proto_rawDescData)\n\t})\n\treturn file_spark_connect_expressions_proto_rawDescData\n}\n\nvar file_spark_connect_expressions_proto_enumTypes = make([]protoimpl.EnumInfo, 6)\nvar file_spark_connect_expressions_proto_msgTypes = make([]protoimpl.MessageInfo, 35)\nvar file_spark_connect_expressions_proto_goTypes = []interface{}{\n\t(Expression_Window_WindowFrame_FrameType)(0),        // 0: spark.connect.Expression.Window.WindowFrame.FrameType\n\t(Expression_SortOrder_SortDirection)(0),             // 1: spark.connect.Expression.SortOrder.SortDirection\n\t(Expression_SortOrder_NullOrdering)(0),              // 2: spark.connect.Expression.SortOrder.NullOrdering\n\t(Expression_Cast_EvalMode)(0),                       // 3: spark.connect.Expression.Cast.EvalMode\n\t(MergeAction_ActionType)(0),                         // 4: spark.connect.MergeAction.ActionType\n\t(SubqueryExpression_SubqueryType)(0),                // 5: spark.connect.SubqueryExpression.SubqueryType\n\t(*Expression)(nil),                                  // 6: spark.connect.Expression\n\t(*ExpressionCommon)(nil),                            // 7: spark.connect.ExpressionCommon\n\t(*CommonInlineUserDefinedFunction)(nil),             // 8: spark.connect.CommonInlineUserDefinedFunction\n\t(*PythonUDF)(nil),                                   // 9: spark.connect.PythonUDF\n\t(*ScalarScalaUDF)(nil),                              // 10: spark.connect.ScalarScalaUDF\n\t(*JavaUDF)(nil),                                     // 11: spark.connect.JavaUDF\n\t(*TypedAggregateExpression)(nil),                    // 12: spark.connect.TypedAggregateExpression\n\t(*CallFunction)(nil),                                // 13: spark.connect.CallFunction\n\t(*NamedArgumentExpression)(nil),                     // 14: spark.connect.NamedArgumentExpression\n\t(*MergeAction)(nil),                                 // 15: spark.connect.MergeAction\n\t(*SubqueryExpression)(nil),                          // 16: spark.connect.SubqueryExpression\n\t(*Expression_Window)(nil),                           // 17: spark.connect.Expression.Window\n\t(*Expression_SortOrder)(nil),                        // 18: spark.connect.Expression.SortOrder\n\t(*Expression_Cast)(nil),                             // 19: spark.connect.Expression.Cast\n\t(*Expression_Literal)(nil),                          // 20: spark.connect.Expression.Literal\n\t(*Expression_UnresolvedAttribute)(nil),              // 21: spark.connect.Expression.UnresolvedAttribute\n\t(*Expression_UnresolvedFunction)(nil),               // 22: spark.connect.Expression.UnresolvedFunction\n\t(*Expression_ExpressionString)(nil),                 // 23: spark.connect.Expression.ExpressionString\n\t(*Expression_UnresolvedStar)(nil),                   // 24: spark.connect.Expression.UnresolvedStar\n\t(*Expression_UnresolvedRegex)(nil),                  // 25: spark.connect.Expression.UnresolvedRegex\n\t(*Expression_UnresolvedExtractValue)(nil),           // 26: spark.connect.Expression.UnresolvedExtractValue\n\t(*Expression_UpdateFields)(nil),                     // 27: spark.connect.Expression.UpdateFields\n\t(*Expression_Alias)(nil),                            // 28: spark.connect.Expression.Alias\n\t(*Expression_LambdaFunction)(nil),                   // 29: spark.connect.Expression.LambdaFunction\n\t(*Expression_UnresolvedNamedLambdaVariable)(nil),    // 30: spark.connect.Expression.UnresolvedNamedLambdaVariable\n\t(*Expression_Window_WindowFrame)(nil),               // 31: spark.connect.Expression.Window.WindowFrame\n\t(*Expression_Window_WindowFrame_FrameBoundary)(nil), // 32: spark.connect.Expression.Window.WindowFrame.FrameBoundary\n\t(*Expression_Literal_Decimal)(nil),                  // 33: spark.connect.Expression.Literal.Decimal\n\t(*Expression_Literal_CalendarInterval)(nil),         // 34: spark.connect.Expression.Literal.CalendarInterval\n\t(*Expression_Literal_Array)(nil),                    // 35: spark.connect.Expression.Literal.Array\n\t(*Expression_Literal_Map)(nil),                      // 36: spark.connect.Expression.Literal.Map\n\t(*Expression_Literal_Struct)(nil),                   // 37: spark.connect.Expression.Literal.Struct\n\t(*Expression_Literal_SpecializedArray)(nil),         // 38: spark.connect.Expression.Literal.SpecializedArray\n\t(*MergeAction_Assignment)(nil),                      // 39: spark.connect.MergeAction.Assignment\n\t(*SubqueryExpression_TableArgOptions)(nil),          // 40: spark.connect.SubqueryExpression.TableArgOptions\n\t(*anypb.Any)(nil),                                   // 41: google.protobuf.Any\n\t(*Origin)(nil),                                      // 42: spark.connect.Origin\n\t(*DataType)(nil),                                    // 43: spark.connect.DataType\n\t(*Bools)(nil),                                       // 44: spark.connect.Bools\n\t(*Ints)(nil),                                        // 45: spark.connect.Ints\n\t(*Longs)(nil),                                       // 46: spark.connect.Longs\n\t(*Floats)(nil),                                      // 47: spark.connect.Floats\n\t(*Doubles)(nil),                                     // 48: spark.connect.Doubles\n\t(*Strings)(nil),                                     // 49: spark.connect.Strings\n}\nvar file_spark_connect_expressions_proto_depIdxs = []int32{\n\t7,  // 0: spark.connect.Expression.common:type_name -> spark.connect.ExpressionCommon\n\t20, // 1: spark.connect.Expression.literal:type_name -> spark.connect.Expression.Literal\n\t21, // 2: spark.connect.Expression.unresolved_attribute:type_name -> spark.connect.Expression.UnresolvedAttribute\n\t22, // 3: spark.connect.Expression.unresolved_function:type_name -> spark.connect.Expression.UnresolvedFunction\n\t23, // 4: spark.connect.Expression.expression_string:type_name -> spark.connect.Expression.ExpressionString\n\t24, // 5: spark.connect.Expression.unresolved_star:type_name -> spark.connect.Expression.UnresolvedStar\n\t28, // 6: spark.connect.Expression.alias:type_name -> spark.connect.Expression.Alias\n\t19, // 7: spark.connect.Expression.cast:type_name -> spark.connect.Expression.Cast\n\t25, // 8: spark.connect.Expression.unresolved_regex:type_name -> spark.connect.Expression.UnresolvedRegex\n\t18, // 9: spark.connect.Expression.sort_order:type_name -> spark.connect.Expression.SortOrder\n\t29, // 10: spark.connect.Expression.lambda_function:type_name -> spark.connect.Expression.LambdaFunction\n\t17, // 11: spark.connect.Expression.window:type_name -> spark.connect.Expression.Window\n\t26, // 12: spark.connect.Expression.unresolved_extract_value:type_name -> spark.connect.Expression.UnresolvedExtractValue\n\t27, // 13: spark.connect.Expression.update_fields:type_name -> spark.connect.Expression.UpdateFields\n\t30, // 14: spark.connect.Expression.unresolved_named_lambda_variable:type_name -> spark.connect.Expression.UnresolvedNamedLambdaVariable\n\t8,  // 15: spark.connect.Expression.common_inline_user_defined_function:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t13, // 16: spark.connect.Expression.call_function:type_name -> spark.connect.CallFunction\n\t14, // 17: spark.connect.Expression.named_argument_expression:type_name -> spark.connect.NamedArgumentExpression\n\t15, // 18: spark.connect.Expression.merge_action:type_name -> spark.connect.MergeAction\n\t12, // 19: spark.connect.Expression.typed_aggregate_expression:type_name -> spark.connect.TypedAggregateExpression\n\t16, // 20: spark.connect.Expression.subquery_expression:type_name -> spark.connect.SubqueryExpression\n\t41, // 21: spark.connect.Expression.extension:type_name -> google.protobuf.Any\n\t42, // 22: spark.connect.ExpressionCommon.origin:type_name -> spark.connect.Origin\n\t6,  // 23: spark.connect.CommonInlineUserDefinedFunction.arguments:type_name -> spark.connect.Expression\n\t9,  // 24: spark.connect.CommonInlineUserDefinedFunction.python_udf:type_name -> spark.connect.PythonUDF\n\t10, // 25: spark.connect.CommonInlineUserDefinedFunction.scalar_scala_udf:type_name -> spark.connect.ScalarScalaUDF\n\t11, // 26: spark.connect.CommonInlineUserDefinedFunction.java_udf:type_name -> spark.connect.JavaUDF\n\t43, // 27: spark.connect.PythonUDF.output_type:type_name -> spark.connect.DataType\n\t43, // 28: spark.connect.ScalarScalaUDF.inputTypes:type_name -> spark.connect.DataType\n\t43, // 29: spark.connect.ScalarScalaUDF.outputType:type_name -> spark.connect.DataType\n\t43, // 30: spark.connect.JavaUDF.output_type:type_name -> spark.connect.DataType\n\t10, // 31: spark.connect.TypedAggregateExpression.scalar_scala_udf:type_name -> spark.connect.ScalarScalaUDF\n\t6,  // 32: spark.connect.CallFunction.arguments:type_name -> spark.connect.Expression\n\t6,  // 33: spark.connect.NamedArgumentExpression.value:type_name -> spark.connect.Expression\n\t4,  // 34: spark.connect.MergeAction.action_type:type_name -> spark.connect.MergeAction.ActionType\n\t6,  // 35: spark.connect.MergeAction.condition:type_name -> spark.connect.Expression\n\t39, // 36: spark.connect.MergeAction.assignments:type_name -> spark.connect.MergeAction.Assignment\n\t5,  // 37: spark.connect.SubqueryExpression.subquery_type:type_name -> spark.connect.SubqueryExpression.SubqueryType\n\t40, // 38: spark.connect.SubqueryExpression.table_arg_options:type_name -> spark.connect.SubqueryExpression.TableArgOptions\n\t6,  // 39: spark.connect.SubqueryExpression.in_subquery_values:type_name -> spark.connect.Expression\n\t6,  // 40: spark.connect.Expression.Window.window_function:type_name -> spark.connect.Expression\n\t6,  // 41: spark.connect.Expression.Window.partition_spec:type_name -> spark.connect.Expression\n\t18, // 42: spark.connect.Expression.Window.order_spec:type_name -> spark.connect.Expression.SortOrder\n\t31, // 43: spark.connect.Expression.Window.frame_spec:type_name -> spark.connect.Expression.Window.WindowFrame\n\t6,  // 44: spark.connect.Expression.SortOrder.child:type_name -> spark.connect.Expression\n\t1,  // 45: spark.connect.Expression.SortOrder.direction:type_name -> spark.connect.Expression.SortOrder.SortDirection\n\t2,  // 46: spark.connect.Expression.SortOrder.null_ordering:type_name -> spark.connect.Expression.SortOrder.NullOrdering\n\t6,  // 47: spark.connect.Expression.Cast.expr:type_name -> spark.connect.Expression\n\t43, // 48: spark.connect.Expression.Cast.type:type_name -> spark.connect.DataType\n\t3,  // 49: spark.connect.Expression.Cast.eval_mode:type_name -> spark.connect.Expression.Cast.EvalMode\n\t43, // 50: spark.connect.Expression.Literal.null:type_name -> spark.connect.DataType\n\t33, // 51: spark.connect.Expression.Literal.decimal:type_name -> spark.connect.Expression.Literal.Decimal\n\t34, // 52: spark.connect.Expression.Literal.calendar_interval:type_name -> spark.connect.Expression.Literal.CalendarInterval\n\t35, // 53: spark.connect.Expression.Literal.array:type_name -> spark.connect.Expression.Literal.Array\n\t36, // 54: spark.connect.Expression.Literal.map:type_name -> spark.connect.Expression.Literal.Map\n\t37, // 55: spark.connect.Expression.Literal.struct:type_name -> spark.connect.Expression.Literal.Struct\n\t38, // 56: spark.connect.Expression.Literal.specialized_array:type_name -> spark.connect.Expression.Literal.SpecializedArray\n\t6,  // 57: spark.connect.Expression.UnresolvedFunction.arguments:type_name -> spark.connect.Expression\n\t6,  // 58: spark.connect.Expression.UnresolvedExtractValue.child:type_name -> spark.connect.Expression\n\t6,  // 59: spark.connect.Expression.UnresolvedExtractValue.extraction:type_name -> spark.connect.Expression\n\t6,  // 60: spark.connect.Expression.UpdateFields.struct_expression:type_name -> spark.connect.Expression\n\t6,  // 61: spark.connect.Expression.UpdateFields.value_expression:type_name -> spark.connect.Expression\n\t6,  // 62: spark.connect.Expression.Alias.expr:type_name -> spark.connect.Expression\n\t6,  // 63: spark.connect.Expression.LambdaFunction.function:type_name -> spark.connect.Expression\n\t30, // 64: spark.connect.Expression.LambdaFunction.arguments:type_name -> spark.connect.Expression.UnresolvedNamedLambdaVariable\n\t0,  // 65: spark.connect.Expression.Window.WindowFrame.frame_type:type_name -> spark.connect.Expression.Window.WindowFrame.FrameType\n\t32, // 66: spark.connect.Expression.Window.WindowFrame.lower:type_name -> spark.connect.Expression.Window.WindowFrame.FrameBoundary\n\t32, // 67: spark.connect.Expression.Window.WindowFrame.upper:type_name -> spark.connect.Expression.Window.WindowFrame.FrameBoundary\n\t6,  // 68: spark.connect.Expression.Window.WindowFrame.FrameBoundary.value:type_name -> spark.connect.Expression\n\t43, // 69: spark.connect.Expression.Literal.Array.element_type:type_name -> spark.connect.DataType\n\t20, // 70: spark.connect.Expression.Literal.Array.elements:type_name -> spark.connect.Expression.Literal\n\t43, // 71: spark.connect.Expression.Literal.Map.key_type:type_name -> spark.connect.DataType\n\t43, // 72: spark.connect.Expression.Literal.Map.value_type:type_name -> spark.connect.DataType\n\t20, // 73: spark.connect.Expression.Literal.Map.keys:type_name -> spark.connect.Expression.Literal\n\t20, // 74: spark.connect.Expression.Literal.Map.values:type_name -> spark.connect.Expression.Literal\n\t43, // 75: spark.connect.Expression.Literal.Struct.struct_type:type_name -> spark.connect.DataType\n\t20, // 76: spark.connect.Expression.Literal.Struct.elements:type_name -> spark.connect.Expression.Literal\n\t44, // 77: spark.connect.Expression.Literal.SpecializedArray.bools:type_name -> spark.connect.Bools\n\t45, // 78: spark.connect.Expression.Literal.SpecializedArray.ints:type_name -> spark.connect.Ints\n\t46, // 79: spark.connect.Expression.Literal.SpecializedArray.longs:type_name -> spark.connect.Longs\n\t47, // 80: spark.connect.Expression.Literal.SpecializedArray.floats:type_name -> spark.connect.Floats\n\t48, // 81: spark.connect.Expression.Literal.SpecializedArray.doubles:type_name -> spark.connect.Doubles\n\t49, // 82: spark.connect.Expression.Literal.SpecializedArray.strings:type_name -> spark.connect.Strings\n\t6,  // 83: spark.connect.MergeAction.Assignment.key:type_name -> spark.connect.Expression\n\t6,  // 84: spark.connect.MergeAction.Assignment.value:type_name -> spark.connect.Expression\n\t6,  // 85: spark.connect.SubqueryExpression.TableArgOptions.partition_spec:type_name -> spark.connect.Expression\n\t18, // 86: spark.connect.SubqueryExpression.TableArgOptions.order_spec:type_name -> spark.connect.Expression.SortOrder\n\t87, // [87:87] is the sub-list for method output_type\n\t87, // [87:87] is the sub-list for method input_type\n\t87, // [87:87] is the sub-list for extension type_name\n\t87, // [87:87] is the sub-list for extension extendee\n\t0,  // [0:87] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_expressions_proto_init() }\nfunc file_spark_connect_expressions_proto_init() {\n\tif File_spark_connect_expressions_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_types_proto_init()\n\tfile_spark_connect_common_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_expressions_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ExpressionCommon); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CommonInlineUserDefinedFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PythonUDF); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ScalarScalaUDF); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*JavaUDF); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*TypedAggregateExpression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CallFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*NamedArgumentExpression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MergeAction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SubqueryExpression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Window); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_SortOrder); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Cast); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedAttribute); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_ExpressionString); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedStar); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedRegex); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedExtractValue); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UpdateFields); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Alias); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_LambdaFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_UnresolvedNamedLambdaVariable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Window_WindowFrame); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Window_WindowFrame_FrameBoundary); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_Decimal); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_CalendarInterval); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_Array); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_Map); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_Struct); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Expression_Literal_SpecializedArray); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MergeAction_Assignment); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_expressions_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SubqueryExpression_TableArgOptions); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*Expression_Literal_)(nil),\n\t\t(*Expression_UnresolvedAttribute_)(nil),\n\t\t(*Expression_UnresolvedFunction_)(nil),\n\t\t(*Expression_ExpressionString_)(nil),\n\t\t(*Expression_UnresolvedStar_)(nil),\n\t\t(*Expression_Alias_)(nil),\n\t\t(*Expression_Cast_)(nil),\n\t\t(*Expression_UnresolvedRegex_)(nil),\n\t\t(*Expression_SortOrder_)(nil),\n\t\t(*Expression_LambdaFunction_)(nil),\n\t\t(*Expression_Window_)(nil),\n\t\t(*Expression_UnresolvedExtractValue_)(nil),\n\t\t(*Expression_UpdateFields_)(nil),\n\t\t(*Expression_UnresolvedNamedLambdaVariable_)(nil),\n\t\t(*Expression_CommonInlineUserDefinedFunction)(nil),\n\t\t(*Expression_CallFunction)(nil),\n\t\t(*Expression_NamedArgumentExpression)(nil),\n\t\t(*Expression_MergeAction)(nil),\n\t\t(*Expression_TypedAggregateExpression)(nil),\n\t\t(*Expression_SubqueryExpression)(nil),\n\t\t(*Expression_Extension)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[2].OneofWrappers = []interface{}{\n\t\t(*CommonInlineUserDefinedFunction_PythonUdf)(nil),\n\t\t(*CommonInlineUserDefinedFunction_ScalarScalaUdf)(nil),\n\t\t(*CommonInlineUserDefinedFunction_JavaUdf)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[5].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[9].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[10].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[13].OneofWrappers = []interface{}{\n\t\t(*Expression_Cast_Type)(nil),\n\t\t(*Expression_Cast_TypeStr)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[14].OneofWrappers = []interface{}{\n\t\t(*Expression_Literal_Null)(nil),\n\t\t(*Expression_Literal_Binary)(nil),\n\t\t(*Expression_Literal_Boolean)(nil),\n\t\t(*Expression_Literal_Byte)(nil),\n\t\t(*Expression_Literal_Short)(nil),\n\t\t(*Expression_Literal_Integer)(nil),\n\t\t(*Expression_Literal_Long)(nil),\n\t\t(*Expression_Literal_Float)(nil),\n\t\t(*Expression_Literal_Double)(nil),\n\t\t(*Expression_Literal_Decimal_)(nil),\n\t\t(*Expression_Literal_String_)(nil),\n\t\t(*Expression_Literal_Date)(nil),\n\t\t(*Expression_Literal_Timestamp)(nil),\n\t\t(*Expression_Literal_TimestampNtz)(nil),\n\t\t(*Expression_Literal_CalendarInterval_)(nil),\n\t\t(*Expression_Literal_YearMonthInterval)(nil),\n\t\t(*Expression_Literal_DayTimeInterval)(nil),\n\t\t(*Expression_Literal_Array_)(nil),\n\t\t(*Expression_Literal_Map_)(nil),\n\t\t(*Expression_Literal_Struct_)(nil),\n\t\t(*Expression_Literal_SpecializedArray_)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[15].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[16].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[18].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[19].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[22].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[26].OneofWrappers = []interface{}{\n\t\t(*Expression_Window_WindowFrame_FrameBoundary_CurrentRow)(nil),\n\t\t(*Expression_Window_WindowFrame_FrameBoundary_Unbounded)(nil),\n\t\t(*Expression_Window_WindowFrame_FrameBoundary_Value)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[27].OneofWrappers = []interface{}{}\n\tfile_spark_connect_expressions_proto_msgTypes[32].OneofWrappers = []interface{}{\n\t\t(*Expression_Literal_SpecializedArray_Bools)(nil),\n\t\t(*Expression_Literal_SpecializedArray_Ints)(nil),\n\t\t(*Expression_Literal_SpecializedArray_Longs)(nil),\n\t\t(*Expression_Literal_SpecializedArray_Floats)(nil),\n\t\t(*Expression_Literal_SpecializedArray_Doubles)(nil),\n\t\t(*Expression_Literal_SpecializedArray_Strings)(nil),\n\t}\n\tfile_spark_connect_expressions_proto_msgTypes[34].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_expressions_proto_rawDesc,\n\t\t\tNumEnums:      6,\n\t\t\tNumMessages:   35,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_expressions_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_expressions_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_expressions_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_expressions_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_expressions_proto = out.File\n\tfile_spark_connect_expressions_proto_rawDesc = nil\n\tfile_spark_connect_expressions_proto_goTypes = nil\n\tfile_spark_connect_expressions_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/ml.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/ml.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// Command for ML\ntype MlCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Command:\n\t//\n\t//\t*MlCommand_Fit_\n\t//\t*MlCommand_Fetch\n\t//\t*MlCommand_Delete_\n\t//\t*MlCommand_Write_\n\t//\t*MlCommand_Read_\n\t//\t*MlCommand_Evaluate_\n\t//\t*MlCommand_CleanCache_\n\t//\t*MlCommand_GetCacheInfo_\n\tCommand isMlCommand_Command `protobuf_oneof:\"command\"`\n}\n\nfunc (x *MlCommand) Reset() {\n\t*x = MlCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand) ProtoMessage() {}\n\nfunc (x *MlCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand.ProtoReflect.Descriptor instead.\nfunc (*MlCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *MlCommand) GetCommand() isMlCommand_Command {\n\tif m != nil {\n\t\treturn m.Command\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetFit() *MlCommand_Fit {\n\tif x, ok := x.GetCommand().(*MlCommand_Fit_); ok {\n\t\treturn x.Fit\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetFetch() *Fetch {\n\tif x, ok := x.GetCommand().(*MlCommand_Fetch); ok {\n\t\treturn x.Fetch\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetDelete() *MlCommand_Delete {\n\tif x, ok := x.GetCommand().(*MlCommand_Delete_); ok {\n\t\treturn x.Delete\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetWrite() *MlCommand_Write {\n\tif x, ok := x.GetCommand().(*MlCommand_Write_); ok {\n\t\treturn x.Write\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetRead() *MlCommand_Read {\n\tif x, ok := x.GetCommand().(*MlCommand_Read_); ok {\n\t\treturn x.Read\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetEvaluate() *MlCommand_Evaluate {\n\tif x, ok := x.GetCommand().(*MlCommand_Evaluate_); ok {\n\t\treturn x.Evaluate\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetCleanCache() *MlCommand_CleanCache {\n\tif x, ok := x.GetCommand().(*MlCommand_CleanCache_); ok {\n\t\treturn x.CleanCache\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand) GetGetCacheInfo() *MlCommand_GetCacheInfo {\n\tif x, ok := x.GetCommand().(*MlCommand_GetCacheInfo_); ok {\n\t\treturn x.GetCacheInfo\n\t}\n\treturn nil\n}\n\ntype isMlCommand_Command interface {\n\tisMlCommand_Command()\n}\n\ntype MlCommand_Fit_ struct {\n\tFit *MlCommand_Fit `protobuf:\"bytes,1,opt,name=fit,proto3,oneof\"`\n}\n\ntype MlCommand_Fetch struct {\n\tFetch *Fetch `protobuf:\"bytes,2,opt,name=fetch,proto3,oneof\"`\n}\n\ntype MlCommand_Delete_ struct {\n\tDelete *MlCommand_Delete `protobuf:\"bytes,3,opt,name=delete,proto3,oneof\"`\n}\n\ntype MlCommand_Write_ struct {\n\tWrite *MlCommand_Write `protobuf:\"bytes,4,opt,name=write,proto3,oneof\"`\n}\n\ntype MlCommand_Read_ struct {\n\tRead *MlCommand_Read `protobuf:\"bytes,5,opt,name=read,proto3,oneof\"`\n}\n\ntype MlCommand_Evaluate_ struct {\n\tEvaluate *MlCommand_Evaluate `protobuf:\"bytes,6,opt,name=evaluate,proto3,oneof\"`\n}\n\ntype MlCommand_CleanCache_ struct {\n\tCleanCache *MlCommand_CleanCache `protobuf:\"bytes,7,opt,name=clean_cache,json=cleanCache,proto3,oneof\"`\n}\n\ntype MlCommand_GetCacheInfo_ struct {\n\tGetCacheInfo *MlCommand_GetCacheInfo `protobuf:\"bytes,8,opt,name=get_cache_info,json=getCacheInfo,proto3,oneof\"`\n}\n\nfunc (*MlCommand_Fit_) isMlCommand_Command() {}\n\nfunc (*MlCommand_Fetch) isMlCommand_Command() {}\n\nfunc (*MlCommand_Delete_) isMlCommand_Command() {}\n\nfunc (*MlCommand_Write_) isMlCommand_Command() {}\n\nfunc (*MlCommand_Read_) isMlCommand_Command() {}\n\nfunc (*MlCommand_Evaluate_) isMlCommand_Command() {}\n\nfunc (*MlCommand_CleanCache_) isMlCommand_Command() {}\n\nfunc (*MlCommand_GetCacheInfo_) isMlCommand_Command() {}\n\n// The result of MlCommand\ntype MlCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ResultType:\n\t//\n\t//\t*MlCommandResult_Param\n\t//\t*MlCommandResult_Summary\n\t//\t*MlCommandResult_OperatorInfo\n\tResultType isMlCommandResult_ResultType `protobuf_oneof:\"result_type\"`\n}\n\nfunc (x *MlCommandResult) Reset() {\n\t*x = MlCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommandResult) ProtoMessage() {}\n\nfunc (x *MlCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommandResult.ProtoReflect.Descriptor instead.\nfunc (*MlCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (m *MlCommandResult) GetResultType() isMlCommandResult_ResultType {\n\tif m != nil {\n\t\treturn m.ResultType\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommandResult) GetParam() *Expression_Literal {\n\tif x, ok := x.GetResultType().(*MlCommandResult_Param); ok {\n\t\treturn x.Param\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommandResult) GetSummary() string {\n\tif x, ok := x.GetResultType().(*MlCommandResult_Summary); ok {\n\t\treturn x.Summary\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlCommandResult) GetOperatorInfo() *MlCommandResult_MlOperatorInfo {\n\tif x, ok := x.GetResultType().(*MlCommandResult_OperatorInfo); ok {\n\t\treturn x.OperatorInfo\n\t}\n\treturn nil\n}\n\ntype isMlCommandResult_ResultType interface {\n\tisMlCommandResult_ResultType()\n}\n\ntype MlCommandResult_Param struct {\n\t// The result of the attribute\n\tParam *Expression_Literal `protobuf:\"bytes,1,opt,name=param,proto3,oneof\"`\n}\n\ntype MlCommandResult_Summary struct {\n\t// Evaluate a Dataset in a model and return the cached ID of summary\n\tSummary string `protobuf:\"bytes,2,opt,name=summary,proto3,oneof\"`\n}\n\ntype MlCommandResult_OperatorInfo struct {\n\t// Operator information\n\tOperatorInfo *MlCommandResult_MlOperatorInfo `protobuf:\"bytes,3,opt,name=operator_info,json=operatorInfo,proto3,oneof\"`\n}\n\nfunc (*MlCommandResult_Param) isMlCommandResult_ResultType() {}\n\nfunc (*MlCommandResult_Summary) isMlCommandResult_ResultType() {}\n\nfunc (*MlCommandResult_OperatorInfo) isMlCommandResult_ResultType() {}\n\n// Command for estimator.fit(dataset)\ntype MlCommand_Fit struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Estimator information (its type should be OPERATOR_TYPE_ESTIMATOR)\n\tEstimator *MlOperator `protobuf:\"bytes,1,opt,name=estimator,proto3\" json:\"estimator,omitempty\"`\n\t// (Optional) parameters of the Estimator\n\tParams *MlParams `protobuf:\"bytes,2,opt,name=params,proto3,oneof\" json:\"params,omitempty\"`\n\t// (Required) the training dataset\n\tDataset *Relation `protobuf:\"bytes,3,opt,name=dataset,proto3\" json:\"dataset,omitempty\"`\n}\n\nfunc (x *MlCommand_Fit) Reset() {\n\t*x = MlCommand_Fit{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_Fit) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_Fit) ProtoMessage() {}\n\nfunc (x *MlCommand_Fit) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_Fit.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_Fit) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 0}\n}\n\nfunc (x *MlCommand_Fit) GetEstimator() *MlOperator {\n\tif x != nil {\n\t\treturn x.Estimator\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Fit) GetParams() *MlParams {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Fit) GetDataset() *Relation {\n\tif x != nil {\n\t\treturn x.Dataset\n\t}\n\treturn nil\n}\n\n// Command to delete the cached objects which could be a model\n// or summary evaluated by a model\ntype MlCommand_Delete struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tObjRefs []*ObjectRef `protobuf:\"bytes,1,rep,name=obj_refs,json=objRefs,proto3\" json:\"obj_refs,omitempty\"`\n}\n\nfunc (x *MlCommand_Delete) Reset() {\n\t*x = MlCommand_Delete{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_Delete) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_Delete) ProtoMessage() {}\n\nfunc (x *MlCommand_Delete) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_Delete.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_Delete) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 1}\n}\n\nfunc (x *MlCommand_Delete) GetObjRefs() []*ObjectRef {\n\tif x != nil {\n\t\treturn x.ObjRefs\n\t}\n\treturn nil\n}\n\n// Force to clean up all the ML cached objects\ntype MlCommand_CleanCache struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *MlCommand_CleanCache) Reset() {\n\t*x = MlCommand_CleanCache{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_CleanCache) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_CleanCache) ProtoMessage() {}\n\nfunc (x *MlCommand_CleanCache) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_CleanCache.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_CleanCache) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 2}\n}\n\n// Get the information of all the ML cached objects\ntype MlCommand_GetCacheInfo struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *MlCommand_GetCacheInfo) Reset() {\n\t*x = MlCommand_GetCacheInfo{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_GetCacheInfo) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_GetCacheInfo) ProtoMessage() {}\n\nfunc (x *MlCommand_GetCacheInfo) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_GetCacheInfo.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_GetCacheInfo) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 3}\n}\n\n// Command to write ML operator\ntype MlCommand_Write struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// It could be an estimator/evaluator or the cached model\n\t//\n\t// Types that are assignable to Type:\n\t//\n\t//\t*MlCommand_Write_Operator\n\t//\t*MlCommand_Write_ObjRef\n\tType isMlCommand_Write_Type `protobuf_oneof:\"type\"`\n\t// (Optional) The parameters of operator which could be estimator/evaluator or a cached model\n\tParams *MlParams `protobuf:\"bytes,3,opt,name=params,proto3,oneof\" json:\"params,omitempty\"`\n\t// (Required) Save the ML instance to the path\n\tPath string `protobuf:\"bytes,4,opt,name=path,proto3\" json:\"path,omitempty\"`\n\t// (Optional) Overwrites if the output path already exists.\n\tShouldOverwrite *bool `protobuf:\"varint,5,opt,name=should_overwrite,json=shouldOverwrite,proto3,oneof\" json:\"should_overwrite,omitempty\"`\n\t// (Optional) The options of the writer\n\tOptions map[string]string `protobuf:\"bytes,6,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *MlCommand_Write) Reset() {\n\t*x = MlCommand_Write{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_Write) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_Write) ProtoMessage() {}\n\nfunc (x *MlCommand_Write) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_Write.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_Write) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 4}\n}\n\nfunc (m *MlCommand_Write) GetType() isMlCommand_Write_Type {\n\tif m != nil {\n\t\treturn m.Type\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Write) GetOperator() *MlOperator {\n\tif x, ok := x.GetType().(*MlCommand_Write_Operator); ok {\n\t\treturn x.Operator\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Write) GetObjRef() *ObjectRef {\n\tif x, ok := x.GetType().(*MlCommand_Write_ObjRef); ok {\n\t\treturn x.ObjRef\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Write) GetParams() *MlParams {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Write) GetPath() string {\n\tif x != nil {\n\t\treturn x.Path\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlCommand_Write) GetShouldOverwrite() bool {\n\tif x != nil && x.ShouldOverwrite != nil {\n\t\treturn *x.ShouldOverwrite\n\t}\n\treturn false\n}\n\nfunc (x *MlCommand_Write) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\ntype isMlCommand_Write_Type interface {\n\tisMlCommand_Write_Type()\n}\n\ntype MlCommand_Write_Operator struct {\n\t// Estimator or evaluator\n\tOperator *MlOperator `protobuf:\"bytes,1,opt,name=operator,proto3,oneof\"`\n}\n\ntype MlCommand_Write_ObjRef struct {\n\t// The cached model\n\tObjRef *ObjectRef `protobuf:\"bytes,2,opt,name=obj_ref,json=objRef,proto3,oneof\"`\n}\n\nfunc (*MlCommand_Write_Operator) isMlCommand_Write_Type() {}\n\nfunc (*MlCommand_Write_ObjRef) isMlCommand_Write_Type() {}\n\n// Command to load ML operator.\ntype MlCommand_Read struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) ML operator information\n\tOperator *MlOperator `protobuf:\"bytes,1,opt,name=operator,proto3\" json:\"operator,omitempty\"`\n\t// (Required) Load the ML instance from the input path\n\tPath string `protobuf:\"bytes,2,opt,name=path,proto3\" json:\"path,omitempty\"`\n}\n\nfunc (x *MlCommand_Read) Reset() {\n\t*x = MlCommand_Read{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_Read) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_Read) ProtoMessage() {}\n\nfunc (x *MlCommand_Read) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_Read.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_Read) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 5}\n}\n\nfunc (x *MlCommand_Read) GetOperator() *MlOperator {\n\tif x != nil {\n\t\treturn x.Operator\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Read) GetPath() string {\n\tif x != nil {\n\t\treturn x.Path\n\t}\n\treturn \"\"\n}\n\n// Command for evaluator.evaluate(dataset)\ntype MlCommand_Evaluate struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Evaluator information (its type should be OPERATOR_TYPE_EVALUATOR)\n\tEvaluator *MlOperator `protobuf:\"bytes,1,opt,name=evaluator,proto3\" json:\"evaluator,omitempty\"`\n\t// (Optional) parameters of the Evaluator\n\tParams *MlParams `protobuf:\"bytes,2,opt,name=params,proto3,oneof\" json:\"params,omitempty\"`\n\t// (Required) the evaluating dataset\n\tDataset *Relation `protobuf:\"bytes,3,opt,name=dataset,proto3\" json:\"dataset,omitempty\"`\n}\n\nfunc (x *MlCommand_Evaluate) Reset() {\n\t*x = MlCommand_Evaluate{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommand_Evaluate) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommand_Evaluate) ProtoMessage() {}\n\nfunc (x *MlCommand_Evaluate) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommand_Evaluate.ProtoReflect.Descriptor instead.\nfunc (*MlCommand_Evaluate) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{0, 6}\n}\n\nfunc (x *MlCommand_Evaluate) GetEvaluator() *MlOperator {\n\tif x != nil {\n\t\treturn x.Evaluator\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Evaluate) GetParams() *MlParams {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommand_Evaluate) GetDataset() *Relation {\n\tif x != nil {\n\t\treturn x.Dataset\n\t}\n\treturn nil\n}\n\n// Represents an operator info\ntype MlCommandResult_MlOperatorInfo struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Type:\n\t//\n\t//\t*MlCommandResult_MlOperatorInfo_ObjRef\n\t//\t*MlCommandResult_MlOperatorInfo_Name\n\tType isMlCommandResult_MlOperatorInfo_Type `protobuf_oneof:\"type\"`\n\t// (Optional) the 'uid' of a ML object\n\t// Note it is different from the 'id' of a cached object.\n\tUid *string `protobuf:\"bytes,3,opt,name=uid,proto3,oneof\" json:\"uid,omitempty\"`\n\t// (Optional) parameters\n\tParams *MlParams `protobuf:\"bytes,4,opt,name=params,proto3,oneof\" json:\"params,omitempty\"`\n\t// (Optional) warning message generated during the ML command execution\n\tWarningMessage *string `protobuf:\"bytes,5,opt,name=warning_message,json=warningMessage,proto3,oneof\" json:\"warning_message,omitempty\"`\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) Reset() {\n\t*x = MlCommandResult_MlOperatorInfo{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlCommandResult_MlOperatorInfo) ProtoMessage() {}\n\nfunc (x *MlCommandResult_MlOperatorInfo) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlCommandResult_MlOperatorInfo.ProtoReflect.Descriptor instead.\nfunc (*MlCommandResult_MlOperatorInfo) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_proto_rawDescGZIP(), []int{1, 0}\n}\n\nfunc (m *MlCommandResult_MlOperatorInfo) GetType() isMlCommandResult_MlOperatorInfo_Type {\n\tif m != nil {\n\t\treturn m.Type\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) GetObjRef() *ObjectRef {\n\tif x, ok := x.GetType().(*MlCommandResult_MlOperatorInfo_ObjRef); ok {\n\t\treturn x.ObjRef\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) GetName() string {\n\tif x, ok := x.GetType().(*MlCommandResult_MlOperatorInfo_Name); ok {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) GetUid() string {\n\tif x != nil && x.Uid != nil {\n\t\treturn *x.Uid\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) GetParams() *MlParams {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\nfunc (x *MlCommandResult_MlOperatorInfo) GetWarningMessage() string {\n\tif x != nil && x.WarningMessage != nil {\n\t\treturn *x.WarningMessage\n\t}\n\treturn \"\"\n}\n\ntype isMlCommandResult_MlOperatorInfo_Type interface {\n\tisMlCommandResult_MlOperatorInfo_Type()\n}\n\ntype MlCommandResult_MlOperatorInfo_ObjRef struct {\n\t// The cached object which could be a model or summary evaluated by a model\n\tObjRef *ObjectRef `protobuf:\"bytes,1,opt,name=obj_ref,json=objRef,proto3,oneof\"`\n}\n\ntype MlCommandResult_MlOperatorInfo_Name struct {\n\t// Operator name\n\tName string `protobuf:\"bytes,2,opt,name=name,proto3,oneof\"`\n}\n\nfunc (*MlCommandResult_MlOperatorInfo_ObjRef) isMlCommandResult_MlOperatorInfo_Type() {}\n\nfunc (*MlCommandResult_MlOperatorInfo_Name) isMlCommandResult_MlOperatorInfo_Type() {}\n\nvar File_spark_connect_ml_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_ml_proto_rawDesc = []byte{\n\t0x0a, 0x16, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x6d, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x6d, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,\n\t0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xb2, 0x0b, 0x0a, 0x09, 0x4d, 0x6c, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x12, 0x30, 0x0a, 0x03, 0x66, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x46, 0x69, 0x74, 0x48,\n\t0x00, 0x52, 0x03, 0x66, 0x69, 0x74, 0x12, 0x2c, 0x0a, 0x05, 0x66, 0x65, 0x74, 0x63, 0x68, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x48, 0x00, 0x52, 0x05, 0x66,\n\t0x65, 0x74, 0x63, 0x68, 0x12, 0x39, 0x0a, 0x06, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x44,\n\t0x65, 0x6c, 0x65, 0x74, 0x65, 0x48, 0x00, 0x52, 0x06, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12,\n\t0x36, 0x0a, 0x05, 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d,\n\t0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x48, 0x00,\n\t0x52, 0x05, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x33, 0x0a, 0x04, 0x72, 0x65, 0x61, 0x64, 0x18,\n\t0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e,\n\t0x52, 0x65, 0x61, 0x64, 0x48, 0x00, 0x52, 0x04, 0x72, 0x65, 0x61, 0x64, 0x12, 0x3f, 0x0a, 0x08,\n\t0x65, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d,\n\t0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x45, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74,\n\t0x65, 0x48, 0x00, 0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x65, 0x12, 0x46, 0x0a,\n\t0x0b, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x5f, 0x63, 0x61, 0x63, 0x68, 0x65, 0x18, 0x07, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x43, 0x6c, 0x65,\n\t0x61, 0x6e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6c, 0x65, 0x61, 0x6e,\n\t0x43, 0x61, 0x63, 0x68, 0x65, 0x12, 0x4d, 0x0a, 0x0e, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x61, 0x63,\n\t0x68, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x61, 0x63, 0x68, 0x65,\n\t0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x0c, 0x67, 0x65, 0x74, 0x43, 0x61, 0x63, 0x68, 0x65,\n\t0x49, 0x6e, 0x66, 0x6f, 0x1a, 0xb2, 0x01, 0x0a, 0x03, 0x46, 0x69, 0x74, 0x12, 0x37, 0x0a, 0x09,\n\t0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x4d, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x73, 0x74, 0x69,\n\t0x6d, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x34, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x48, 0x00,\n\t0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x88, 0x01, 0x01, 0x12, 0x31, 0x0a, 0x07, 0x64,\n\t0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x42, 0x09,\n\t0x0a, 0x07, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x3d, 0x0a, 0x06, 0x44, 0x65, 0x6c,\n\t0x65, 0x74, 0x65, 0x12, 0x33, 0x0a, 0x08, 0x6f, 0x62, 0x6a, 0x5f, 0x72, 0x65, 0x66, 0x73, 0x18,\n\t0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x66, 0x52,\n\t0x07, 0x6f, 0x62, 0x6a, 0x52, 0x65, 0x66, 0x73, 0x1a, 0x0c, 0x0a, 0x0a, 0x43, 0x6c, 0x65, 0x61,\n\t0x6e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x43, 0x61, 0x63,\n\t0x68, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x9a, 0x03, 0x0a, 0x05, 0x57, 0x72, 0x69, 0x74, 0x65,\n\t0x12, 0x37, 0x0a, 0x08, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x48, 0x00, 0x52,\n\t0x08, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x33, 0x0a, 0x07, 0x6f, 0x62, 0x6a,\n\t0x5f, 0x72, 0x65, 0x66, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63,\n\t0x74, 0x52, 0x65, 0x66, 0x48, 0x00, 0x52, 0x06, 0x6f, 0x62, 0x6a, 0x52, 0x65, 0x66, 0x12, 0x34,\n\t0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d,\n\t0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x48, 0x01, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d,\n\t0x73, 0x88, 0x01, 0x01, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x04, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x2e, 0x0a, 0x10, 0x73, 0x68, 0x6f, 0x75,\n\t0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01,\n\t0x28, 0x08, 0x48, 0x02, 0x52, 0x0f, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x4f, 0x76, 0x65, 0x72,\n\t0x77, 0x72, 0x69, 0x74, 0x65, 0x88, 0x01, 0x01, 0x12, 0x45, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x2e, 0x57, 0x72, 0x69, 0x74, 0x65, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,\n\t0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a,\n\t0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,\n\t0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,\n\t0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x06, 0x0a, 0x04, 0x74,\n\t0x79, 0x70, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x42, 0x13,\n\t0x0a, 0x11, 0x5f, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x77, 0x72,\n\t0x69, 0x74, 0x65, 0x1a, 0x51, 0x0a, 0x04, 0x52, 0x65, 0x61, 0x64, 0x12, 0x35, 0x0a, 0x08, 0x6f,\n\t0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c,\n\t0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x08, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74,\n\t0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x1a, 0xb7, 0x01, 0x0a, 0x08, 0x45, 0x76, 0x61, 0x6c, 0x75,\n\t0x61, 0x74, 0x65, 0x12, 0x37, 0x0a, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x6f, 0x72,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f,\n\t0x72, 0x52, 0x09, 0x65, 0x76, 0x61, 0x6c, 0x75, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x34, 0x0a, 0x06,\n\t0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x50,\n\t0x61, 0x72, 0x61, 0x6d, 0x73, 0x48, 0x00, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x88,\n\t0x01, 0x01, 0x12, 0x31, 0x0a, 0x07, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x64, 0x61,\n\t0x74, 0x61, 0x73, 0x65, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73,\n\t0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x22, 0xd5, 0x03, 0x0a, 0x0f,\n\t0x4d, 0x6c, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12,\n\t0x39, 0x0a, 0x05, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61,\n\t0x6c, 0x48, 0x00, 0x52, 0x05, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x12, 0x1a, 0x0a, 0x07, 0x73, 0x75,\n\t0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x07, 0x73,\n\t0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x54, 0x0a, 0x0d, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74,\n\t0x6f, 0x72, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c,\n\t0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x4d, 0x6c,\n\t0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x0c,\n\t0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x85, 0x02, 0x0a,\n\t0x0e, 0x4d, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12,\n\t0x33, 0x0a, 0x07, 0x6f, 0x62, 0x6a, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x66, 0x48, 0x00, 0x52, 0x06, 0x6f, 0x62,\n\t0x6a, 0x52, 0x65, 0x66, 0x12, 0x14, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x00, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x15, 0x0a, 0x03, 0x75, 0x69,\n\t0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x03, 0x75, 0x69, 0x64, 0x88, 0x01,\n\t0x01, 0x12, 0x34, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x4d, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x48, 0x02, 0x52, 0x06, 0x70, 0x61,\n\t0x72, 0x61, 0x6d, 0x73, 0x88, 0x01, 0x01, 0x12, 0x2c, 0x0a, 0x0f, 0x77, 0x61, 0x72, 0x6e, 0x69,\n\t0x6e, 0x67, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x03, 0x52, 0x0e, 0x77, 0x61, 0x72, 0x6e, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x73, 0x73, 0x61,\n\t0x67, 0x65, 0x88, 0x01, 0x01, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x42, 0x06, 0x0a,\n\t0x04, 0x5f, 0x75, 0x69, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73,\n\t0x42, 0x12, 0x0a, 0x10, 0x5f, 0x77, 0x61, 0x72, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x65, 0x73,\n\t0x73, 0x61, 0x67, 0x65, 0x42, 0x0d, 0x0a, 0x0b, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,\n\t0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,\n\t0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_ml_proto_rawDescOnce sync.Once\n\tfile_spark_connect_ml_proto_rawDescData = file_spark_connect_ml_proto_rawDesc\n)\n\nfunc file_spark_connect_ml_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_ml_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_ml_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_ml_proto_rawDescData)\n\t})\n\treturn file_spark_connect_ml_proto_rawDescData\n}\n\nvar file_spark_connect_ml_proto_msgTypes = make([]protoimpl.MessageInfo, 11)\nvar file_spark_connect_ml_proto_goTypes = []interface{}{\n\t(*MlCommand)(nil),                      // 0: spark.connect.MlCommand\n\t(*MlCommandResult)(nil),                // 1: spark.connect.MlCommandResult\n\t(*MlCommand_Fit)(nil),                  // 2: spark.connect.MlCommand.Fit\n\t(*MlCommand_Delete)(nil),               // 3: spark.connect.MlCommand.Delete\n\t(*MlCommand_CleanCache)(nil),           // 4: spark.connect.MlCommand.CleanCache\n\t(*MlCommand_GetCacheInfo)(nil),         // 5: spark.connect.MlCommand.GetCacheInfo\n\t(*MlCommand_Write)(nil),                // 6: spark.connect.MlCommand.Write\n\t(*MlCommand_Read)(nil),                 // 7: spark.connect.MlCommand.Read\n\t(*MlCommand_Evaluate)(nil),             // 8: spark.connect.MlCommand.Evaluate\n\tnil,                                    // 9: spark.connect.MlCommand.Write.OptionsEntry\n\t(*MlCommandResult_MlOperatorInfo)(nil), // 10: spark.connect.MlCommandResult.MlOperatorInfo\n\t(*Fetch)(nil),                          // 11: spark.connect.Fetch\n\t(*Expression_Literal)(nil),             // 12: spark.connect.Expression.Literal\n\t(*MlOperator)(nil),                     // 13: spark.connect.MlOperator\n\t(*MlParams)(nil),                       // 14: spark.connect.MlParams\n\t(*Relation)(nil),                       // 15: spark.connect.Relation\n\t(*ObjectRef)(nil),                      // 16: spark.connect.ObjectRef\n}\nvar file_spark_connect_ml_proto_depIdxs = []int32{\n\t2,  // 0: spark.connect.MlCommand.fit:type_name -> spark.connect.MlCommand.Fit\n\t11, // 1: spark.connect.MlCommand.fetch:type_name -> spark.connect.Fetch\n\t3,  // 2: spark.connect.MlCommand.delete:type_name -> spark.connect.MlCommand.Delete\n\t6,  // 3: spark.connect.MlCommand.write:type_name -> spark.connect.MlCommand.Write\n\t7,  // 4: spark.connect.MlCommand.read:type_name -> spark.connect.MlCommand.Read\n\t8,  // 5: spark.connect.MlCommand.evaluate:type_name -> spark.connect.MlCommand.Evaluate\n\t4,  // 6: spark.connect.MlCommand.clean_cache:type_name -> spark.connect.MlCommand.CleanCache\n\t5,  // 7: spark.connect.MlCommand.get_cache_info:type_name -> spark.connect.MlCommand.GetCacheInfo\n\t12, // 8: spark.connect.MlCommandResult.param:type_name -> spark.connect.Expression.Literal\n\t10, // 9: spark.connect.MlCommandResult.operator_info:type_name -> spark.connect.MlCommandResult.MlOperatorInfo\n\t13, // 10: spark.connect.MlCommand.Fit.estimator:type_name -> spark.connect.MlOperator\n\t14, // 11: spark.connect.MlCommand.Fit.params:type_name -> spark.connect.MlParams\n\t15, // 12: spark.connect.MlCommand.Fit.dataset:type_name -> spark.connect.Relation\n\t16, // 13: spark.connect.MlCommand.Delete.obj_refs:type_name -> spark.connect.ObjectRef\n\t13, // 14: spark.connect.MlCommand.Write.operator:type_name -> spark.connect.MlOperator\n\t16, // 15: spark.connect.MlCommand.Write.obj_ref:type_name -> spark.connect.ObjectRef\n\t14, // 16: spark.connect.MlCommand.Write.params:type_name -> spark.connect.MlParams\n\t9,  // 17: spark.connect.MlCommand.Write.options:type_name -> spark.connect.MlCommand.Write.OptionsEntry\n\t13, // 18: spark.connect.MlCommand.Read.operator:type_name -> spark.connect.MlOperator\n\t13, // 19: spark.connect.MlCommand.Evaluate.evaluator:type_name -> spark.connect.MlOperator\n\t14, // 20: spark.connect.MlCommand.Evaluate.params:type_name -> spark.connect.MlParams\n\t15, // 21: spark.connect.MlCommand.Evaluate.dataset:type_name -> spark.connect.Relation\n\t16, // 22: spark.connect.MlCommandResult.MlOperatorInfo.obj_ref:type_name -> spark.connect.ObjectRef\n\t14, // 23: spark.connect.MlCommandResult.MlOperatorInfo.params:type_name -> spark.connect.MlParams\n\t24, // [24:24] is the sub-list for method output_type\n\t24, // [24:24] is the sub-list for method input_type\n\t24, // [24:24] is the sub-list for extension type_name\n\t24, // [24:24] is the sub-list for extension extendee\n\t0,  // [0:24] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_ml_proto_init() }\nfunc file_spark_connect_ml_proto_init() {\n\tif File_spark_connect_ml_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_relations_proto_init()\n\tfile_spark_connect_expressions_proto_init()\n\tfile_spark_connect_ml_common_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_ml_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_Fit); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_Delete); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_CleanCache); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_GetCacheInfo); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_Write); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_Read); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommand_Evaluate); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlCommandResult_MlOperatorInfo); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_ml_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*MlCommand_Fit_)(nil),\n\t\t(*MlCommand_Fetch)(nil),\n\t\t(*MlCommand_Delete_)(nil),\n\t\t(*MlCommand_Write_)(nil),\n\t\t(*MlCommand_Read_)(nil),\n\t\t(*MlCommand_Evaluate_)(nil),\n\t\t(*MlCommand_CleanCache_)(nil),\n\t\t(*MlCommand_GetCacheInfo_)(nil),\n\t}\n\tfile_spark_connect_ml_proto_msgTypes[1].OneofWrappers = []interface{}{\n\t\t(*MlCommandResult_Param)(nil),\n\t\t(*MlCommandResult_Summary)(nil),\n\t\t(*MlCommandResult_OperatorInfo)(nil),\n\t}\n\tfile_spark_connect_ml_proto_msgTypes[2].OneofWrappers = []interface{}{}\n\tfile_spark_connect_ml_proto_msgTypes[6].OneofWrappers = []interface{}{\n\t\t(*MlCommand_Write_Operator)(nil),\n\t\t(*MlCommand_Write_ObjRef)(nil),\n\t}\n\tfile_spark_connect_ml_proto_msgTypes[8].OneofWrappers = []interface{}{}\n\tfile_spark_connect_ml_proto_msgTypes[10].OneofWrappers = []interface{}{\n\t\t(*MlCommandResult_MlOperatorInfo_ObjRef)(nil),\n\t\t(*MlCommandResult_MlOperatorInfo_Name)(nil),\n\t}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_ml_proto_rawDesc,\n\t\t\tNumEnums:      0,\n\t\t\tNumMessages:   11,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_ml_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_ml_proto_depIdxs,\n\t\tMessageInfos:      file_spark_connect_ml_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_ml_proto = out.File\n\tfile_spark_connect_ml_proto_rawDesc = nil\n\tfile_spark_connect_ml_proto_goTypes = nil\n\tfile_spark_connect_ml_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/ml_common.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/ml_common.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\ntype MlOperator_OperatorType int32\n\nconst (\n\tMlOperator_OPERATOR_TYPE_UNSPECIFIED MlOperator_OperatorType = 0\n\t// ML estimator\n\tMlOperator_OPERATOR_TYPE_ESTIMATOR MlOperator_OperatorType = 1\n\t// ML transformer (non-model)\n\tMlOperator_OPERATOR_TYPE_TRANSFORMER MlOperator_OperatorType = 2\n\t// ML evaluator\n\tMlOperator_OPERATOR_TYPE_EVALUATOR MlOperator_OperatorType = 3\n\t// ML model\n\tMlOperator_OPERATOR_TYPE_MODEL MlOperator_OperatorType = 4\n)\n\n// Enum value maps for MlOperator_OperatorType.\nvar (\n\tMlOperator_OperatorType_name = map[int32]string{\n\t\t0: \"OPERATOR_TYPE_UNSPECIFIED\",\n\t\t1: \"OPERATOR_TYPE_ESTIMATOR\",\n\t\t2: \"OPERATOR_TYPE_TRANSFORMER\",\n\t\t3: \"OPERATOR_TYPE_EVALUATOR\",\n\t\t4: \"OPERATOR_TYPE_MODEL\",\n\t}\n\tMlOperator_OperatorType_value = map[string]int32{\n\t\t\"OPERATOR_TYPE_UNSPECIFIED\": 0,\n\t\t\"OPERATOR_TYPE_ESTIMATOR\":   1,\n\t\t\"OPERATOR_TYPE_TRANSFORMER\": 2,\n\t\t\"OPERATOR_TYPE_EVALUATOR\":   3,\n\t\t\"OPERATOR_TYPE_MODEL\":       4,\n\t}\n)\n\nfunc (x MlOperator_OperatorType) Enum() *MlOperator_OperatorType {\n\tp := new(MlOperator_OperatorType)\n\t*p = x\n\treturn p\n}\n\nfunc (x MlOperator_OperatorType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (MlOperator_OperatorType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_ml_common_proto_enumTypes[0].Descriptor()\n}\n\nfunc (MlOperator_OperatorType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_ml_common_proto_enumTypes[0]\n}\n\nfunc (x MlOperator_OperatorType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use MlOperator_OperatorType.Descriptor instead.\nfunc (MlOperator_OperatorType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_common_proto_rawDescGZIP(), []int{1, 0}\n}\n\n// MlParams stores param settings for ML Estimator / Transformer / Evaluator\ntype MlParams struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// User-supplied params\n\tParams map[string]*Expression_Literal `protobuf:\"bytes,1,rep,name=params,proto3\" json:\"params,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *MlParams) Reset() {\n\t*x = MlParams{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_common_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlParams) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlParams) ProtoMessage() {}\n\nfunc (x *MlParams) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_common_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlParams.ProtoReflect.Descriptor instead.\nfunc (*MlParams) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_common_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (x *MlParams) GetParams() map[string]*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\n// MLOperator represents the ML operators like (Estimator, Transformer or Evaluator)\ntype MlOperator struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The qualified name of the ML operator.\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Required) Unique id of the ML operator\n\tUid string `protobuf:\"bytes,2,opt,name=uid,proto3\" json:\"uid,omitempty\"`\n\t// (Required) Represents what the ML operator is\n\tType MlOperator_OperatorType `protobuf:\"varint,3,opt,name=type,proto3,enum=spark.connect.MlOperator_OperatorType\" json:\"type,omitempty\"`\n}\n\nfunc (x *MlOperator) Reset() {\n\t*x = MlOperator{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_common_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlOperator) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlOperator) ProtoMessage() {}\n\nfunc (x *MlOperator) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_common_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlOperator.ProtoReflect.Descriptor instead.\nfunc (*MlOperator) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_common_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *MlOperator) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlOperator) GetUid() string {\n\tif x != nil {\n\t\treturn x.Uid\n\t}\n\treturn \"\"\n}\n\nfunc (x *MlOperator) GetType() MlOperator_OperatorType {\n\tif x != nil {\n\t\treturn x.Type\n\t}\n\treturn MlOperator_OPERATOR_TYPE_UNSPECIFIED\n}\n\n// Represents a reference to the cached object which could be a model\n// or summary evaluated by a model\ntype ObjectRef struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The ID is used to lookup the object on the server side.\n\t// Note it is different from the 'uid' of a ML object.\n\tId string `protobuf:\"bytes,1,opt,name=id,proto3\" json:\"id,omitempty\"`\n}\n\nfunc (x *ObjectRef) Reset() {\n\t*x = ObjectRef{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_ml_common_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ObjectRef) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ObjectRef) ProtoMessage() {}\n\nfunc (x *ObjectRef) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_ml_common_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ObjectRef.ProtoReflect.Descriptor instead.\nfunc (*ObjectRef) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_ml_common_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *ObjectRef) GetId() string {\n\tif x != nil {\n\t\treturn x.Id\n\t}\n\treturn \"\"\n}\n\nvar File_spark_connect_ml_common_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_ml_common_proto_rawDesc = []byte{\n\t0x0a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x6d, 0x6c, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,\n\t0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x1f,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x65, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22,\n\t0xa5, 0x01, 0x0a, 0x08, 0x4d, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x12, 0x3b, 0x0a, 0x06,\n\t0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x50,\n\t0x61, 0x72, 0x61, 0x6d, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72,\n\t0x79, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x5c, 0x0a, 0x0b, 0x50, 0x61, 0x72,\n\t0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x90, 0x02, 0x0a, 0x0a, 0x4d, 0x6c, 0x4f, 0x70,\n\t0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x69,\n\t0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x69, 0x64, 0x12, 0x3a, 0x0a, 0x04,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x4f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x54, 0x79,\n\t0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x9f, 0x01, 0x0a, 0x0c, 0x4f, 0x70, 0x65,\n\t0x72, 0x61, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1d, 0x0a, 0x19, 0x4f, 0x50, 0x45,\n\t0x52, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45,\n\t0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x4f, 0x50, 0x45, 0x52,\n\t0x41, 0x54, 0x4f, 0x52, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x53, 0x54, 0x49, 0x4d, 0x41,\n\t0x54, 0x4f, 0x52, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x4f,\n\t0x52, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d,\n\t0x45, 0x52, 0x10, 0x02, 0x12, 0x1b, 0x0a, 0x17, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52,\n\t0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x56, 0x41, 0x4c, 0x55, 0x41, 0x54, 0x4f, 0x52, 0x10,\n\t0x03, 0x12, 0x17, 0x0a, 0x13, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x54, 0x59,\n\t0x50, 0x45, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x4c, 0x10, 0x04, 0x22, 0x1b, 0x0a, 0x09, 0x4f, 0x62,\n\t0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x66, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61,\n\t0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74,\n\t0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62,\n\t0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_ml_common_proto_rawDescOnce sync.Once\n\tfile_spark_connect_ml_common_proto_rawDescData = file_spark_connect_ml_common_proto_rawDesc\n)\n\nfunc file_spark_connect_ml_common_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_ml_common_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_ml_common_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_ml_common_proto_rawDescData)\n\t})\n\treturn file_spark_connect_ml_common_proto_rawDescData\n}\n\nvar file_spark_connect_ml_common_proto_enumTypes = make([]protoimpl.EnumInfo, 1)\nvar file_spark_connect_ml_common_proto_msgTypes = make([]protoimpl.MessageInfo, 4)\nvar file_spark_connect_ml_common_proto_goTypes = []interface{}{\n\t(MlOperator_OperatorType)(0), // 0: spark.connect.MlOperator.OperatorType\n\t(*MlParams)(nil),             // 1: spark.connect.MlParams\n\t(*MlOperator)(nil),           // 2: spark.connect.MlOperator\n\t(*ObjectRef)(nil),            // 3: spark.connect.ObjectRef\n\tnil,                          // 4: spark.connect.MlParams.ParamsEntry\n\t(*Expression_Literal)(nil),   // 5: spark.connect.Expression.Literal\n}\nvar file_spark_connect_ml_common_proto_depIdxs = []int32{\n\t4, // 0: spark.connect.MlParams.params:type_name -> spark.connect.MlParams.ParamsEntry\n\t0, // 1: spark.connect.MlOperator.type:type_name -> spark.connect.MlOperator.OperatorType\n\t5, // 2: spark.connect.MlParams.ParamsEntry.value:type_name -> spark.connect.Expression.Literal\n\t3, // [3:3] is the sub-list for method output_type\n\t3, // [3:3] is the sub-list for method input_type\n\t3, // [3:3] is the sub-list for extension type_name\n\t3, // [3:3] is the sub-list for extension extendee\n\t0, // [0:3] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_ml_common_proto_init() }\nfunc file_spark_connect_ml_common_proto_init() {\n\tif File_spark_connect_ml_common_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_expressions_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_ml_common_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlParams); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_common_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlOperator); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_ml_common_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ObjectRef); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_ml_common_proto_rawDesc,\n\t\t\tNumEnums:      1,\n\t\t\tNumMessages:   4,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_ml_common_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_ml_common_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_ml_common_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_ml_common_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_ml_common_proto = out.File\n\tfile_spark_connect_ml_common_proto_rawDesc = nil\n\tfile_spark_connect_ml_common_proto_goTypes = nil\n\tfile_spark_connect_ml_common_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/pipelines.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/pipelines.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// The type of dataset.\ntype DatasetType int32\n\nconst (\n\t// Safe default value. Should not be used.\n\tDatasetType_DATASET_TYPE_UNSPECIFIED DatasetType = 0\n\t// A materialized view dataset which is published to the catalog\n\tDatasetType_MATERIALIZED_VIEW DatasetType = 1\n\t// A table which is published to the catalog\n\tDatasetType_TABLE DatasetType = 2\n\t// A view which is not published to the catalog\n\tDatasetType_TEMPORARY_VIEW DatasetType = 3\n)\n\n// Enum value maps for DatasetType.\nvar (\n\tDatasetType_name = map[int32]string{\n\t\t0: \"DATASET_TYPE_UNSPECIFIED\",\n\t\t1: \"MATERIALIZED_VIEW\",\n\t\t2: \"TABLE\",\n\t\t3: \"TEMPORARY_VIEW\",\n\t}\n\tDatasetType_value = map[string]int32{\n\t\t\"DATASET_TYPE_UNSPECIFIED\": 0,\n\t\t\"MATERIALIZED_VIEW\":        1,\n\t\t\"TABLE\":                    2,\n\t\t\"TEMPORARY_VIEW\":           3,\n\t}\n)\n\nfunc (x DatasetType) Enum() *DatasetType {\n\tp := new(DatasetType)\n\t*p = x\n\treturn p\n}\n\nfunc (x DatasetType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (DatasetType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_pipelines_proto_enumTypes[0].Descriptor()\n}\n\nfunc (DatasetType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_pipelines_proto_enumTypes[0]\n}\n\nfunc (x DatasetType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use DatasetType.Descriptor instead.\nfunc (DatasetType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0}\n}\n\n// Dispatch object for pipelines commands. See each individual command for documentation.\ntype PipelineCommand struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to CommandType:\n\t//\n\t//\t*PipelineCommand_CreateDataflowGraph_\n\t//\t*PipelineCommand_DefineDataset_\n\t//\t*PipelineCommand_DefineFlow_\n\t//\t*PipelineCommand_DropDataflowGraph_\n\t//\t*PipelineCommand_StartRun_\n\t//\t*PipelineCommand_DefineSqlGraphElements\n\tCommandType isPipelineCommand_CommandType `protobuf_oneof:\"command_type\"`\n}\n\nfunc (x *PipelineCommand) Reset() {\n\t*x = PipelineCommand{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand) ProtoMessage() {}\n\nfunc (x *PipelineCommand) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *PipelineCommand) GetCommandType() isPipelineCommand_CommandType {\n\tif m != nil {\n\t\treturn m.CommandType\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetCreateDataflowGraph() *PipelineCommand_CreateDataflowGraph {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_CreateDataflowGraph_); ok {\n\t\treturn x.CreateDataflowGraph\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetDefineDataset() *PipelineCommand_DefineDataset {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_DefineDataset_); ok {\n\t\treturn x.DefineDataset\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetDefineFlow() *PipelineCommand_DefineFlow {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_DefineFlow_); ok {\n\t\treturn x.DefineFlow\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetDropDataflowGraph() *PipelineCommand_DropDataflowGraph {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_DropDataflowGraph_); ok {\n\t\treturn x.DropDataflowGraph\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetStartRun() *PipelineCommand_StartRun {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_StartRun_); ok {\n\t\treturn x.StartRun\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand) GetDefineSqlGraphElements() *DefineSqlGraphElements {\n\tif x, ok := x.GetCommandType().(*PipelineCommand_DefineSqlGraphElements); ok {\n\t\treturn x.DefineSqlGraphElements\n\t}\n\treturn nil\n}\n\ntype isPipelineCommand_CommandType interface {\n\tisPipelineCommand_CommandType()\n}\n\ntype PipelineCommand_CreateDataflowGraph_ struct {\n\tCreateDataflowGraph *PipelineCommand_CreateDataflowGraph `protobuf:\"bytes,1,opt,name=create_dataflow_graph,json=createDataflowGraph,proto3,oneof\"`\n}\n\ntype PipelineCommand_DefineDataset_ struct {\n\tDefineDataset *PipelineCommand_DefineDataset `protobuf:\"bytes,2,opt,name=define_dataset,json=defineDataset,proto3,oneof\"`\n}\n\ntype PipelineCommand_DefineFlow_ struct {\n\tDefineFlow *PipelineCommand_DefineFlow `protobuf:\"bytes,3,opt,name=define_flow,json=defineFlow,proto3,oneof\"`\n}\n\ntype PipelineCommand_DropDataflowGraph_ struct {\n\tDropDataflowGraph *PipelineCommand_DropDataflowGraph `protobuf:\"bytes,4,opt,name=drop_dataflow_graph,json=dropDataflowGraph,proto3,oneof\"`\n}\n\ntype PipelineCommand_StartRun_ struct {\n\tStartRun *PipelineCommand_StartRun `protobuf:\"bytes,5,opt,name=start_run,json=startRun,proto3,oneof\"`\n}\n\ntype PipelineCommand_DefineSqlGraphElements struct {\n\tDefineSqlGraphElements *DefineSqlGraphElements `protobuf:\"bytes,6,opt,name=define_sql_graph_elements,json=defineSqlGraphElements,proto3,oneof\"`\n}\n\nfunc (*PipelineCommand_CreateDataflowGraph_) isPipelineCommand_CommandType() {}\n\nfunc (*PipelineCommand_DefineDataset_) isPipelineCommand_CommandType() {}\n\nfunc (*PipelineCommand_DefineFlow_) isPipelineCommand_CommandType() {}\n\nfunc (*PipelineCommand_DropDataflowGraph_) isPipelineCommand_CommandType() {}\n\nfunc (*PipelineCommand_StartRun_) isPipelineCommand_CommandType() {}\n\nfunc (*PipelineCommand_DefineSqlGraphElements) isPipelineCommand_CommandType() {}\n\n// Parses the SQL file and registers all datasets and flows.\ntype DefineSqlGraphElements struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The graph to attach this dataset to.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n\t// The full path to the SQL file. Can be relative or absolute.\n\tSqlFilePath *string `protobuf:\"bytes,2,opt,name=sql_file_path,json=sqlFilePath,proto3,oneof\" json:\"sql_file_path,omitempty\"`\n\t// The contents of the SQL file.\n\tSqlText *string `protobuf:\"bytes,3,opt,name=sql_text,json=sqlText,proto3,oneof\" json:\"sql_text,omitempty\"`\n}\n\nfunc (x *DefineSqlGraphElements) Reset() {\n\t*x = DefineSqlGraphElements{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DefineSqlGraphElements) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DefineSqlGraphElements) ProtoMessage() {}\n\nfunc (x *DefineSqlGraphElements) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DefineSqlGraphElements.ProtoReflect.Descriptor instead.\nfunc (*DefineSqlGraphElements) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (x *DefineSqlGraphElements) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\nfunc (x *DefineSqlGraphElements) GetSqlFilePath() string {\n\tif x != nil && x.SqlFilePath != nil {\n\t\treturn *x.SqlFilePath\n\t}\n\treturn \"\"\n}\n\nfunc (x *DefineSqlGraphElements) GetSqlText() string {\n\tif x != nil && x.SqlText != nil {\n\t\treturn *x.SqlText\n\t}\n\treturn \"\"\n}\n\n// Dispatch object for pipelines command results.\ntype PipelineCommandResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ResultType:\n\t//\n\t//\t*PipelineCommandResult_CreateDataflowGraphResult_\n\tResultType isPipelineCommandResult_ResultType `protobuf_oneof:\"result_type\"`\n}\n\nfunc (x *PipelineCommandResult) Reset() {\n\t*x = PipelineCommandResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommandResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommandResult) ProtoMessage() {}\n\nfunc (x *PipelineCommandResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommandResult.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommandResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (m *PipelineCommandResult) GetResultType() isPipelineCommandResult_ResultType {\n\tif m != nil {\n\t\treturn m.ResultType\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommandResult) GetCreateDataflowGraphResult() *PipelineCommandResult_CreateDataflowGraphResult {\n\tif x, ok := x.GetResultType().(*PipelineCommandResult_CreateDataflowGraphResult_); ok {\n\t\treturn x.CreateDataflowGraphResult\n\t}\n\treturn nil\n}\n\ntype isPipelineCommandResult_ResultType interface {\n\tisPipelineCommandResult_ResultType()\n}\n\ntype PipelineCommandResult_CreateDataflowGraphResult_ struct {\n\tCreateDataflowGraphResult *PipelineCommandResult_CreateDataflowGraphResult `protobuf:\"bytes,1,opt,name=create_dataflow_graph_result,json=createDataflowGraphResult,proto3,oneof\"`\n}\n\nfunc (*PipelineCommandResult_CreateDataflowGraphResult_) isPipelineCommandResult_ResultType() {}\n\n// A response containing an event emitted during the run of a pipeline.\ntype PipelineEventResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tEvent *PipelineEvent `protobuf:\"bytes,1,opt,name=event,proto3\" json:\"event,omitempty\"`\n}\n\nfunc (x *PipelineEventResult) Reset() {\n\t*x = PipelineEventResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineEventResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineEventResult) ProtoMessage() {}\n\nfunc (x *PipelineEventResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineEventResult.ProtoReflect.Descriptor instead.\nfunc (*PipelineEventResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{3}\n}\n\nfunc (x *PipelineEventResult) GetEvent() *PipelineEvent {\n\tif x != nil {\n\t\treturn x.Event\n\t}\n\treturn nil\n}\n\n// An event emitted during the run of a graph.\ntype PipelineEvent struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The time of the event.\n\tTimestamp *string `protobuf:\"bytes,1,opt,name=timestamp,proto3,oneof\" json:\"timestamp,omitempty\"`\n\t// The message that should be displayed to users.\n\tMessage *string `protobuf:\"bytes,2,opt,name=message,proto3,oneof\" json:\"message,omitempty\"`\n}\n\nfunc (x *PipelineEvent) Reset() {\n\t*x = PipelineEvent{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineEvent) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineEvent) ProtoMessage() {}\n\nfunc (x *PipelineEvent) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineEvent.ProtoReflect.Descriptor instead.\nfunc (*PipelineEvent) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{4}\n}\n\nfunc (x *PipelineEvent) GetTimestamp() string {\n\tif x != nil && x.Timestamp != nil {\n\t\treturn *x.Timestamp\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineEvent) GetMessage() string {\n\tif x != nil && x.Message != nil {\n\t\treturn *x.Message\n\t}\n\treturn \"\"\n}\n\n// Request to create a new dataflow graph.\ntype PipelineCommand_CreateDataflowGraph struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The default catalog.\n\tDefaultCatalog *string `protobuf:\"bytes,1,opt,name=default_catalog,json=defaultCatalog,proto3,oneof\" json:\"default_catalog,omitempty\"`\n\t// The default database.\n\tDefaultDatabase *string `protobuf:\"bytes,2,opt,name=default_database,json=defaultDatabase,proto3,oneof\" json:\"default_database,omitempty\"`\n\t// SQL configurations for all flows in this graph.\n\tSqlConf map[string]string `protobuf:\"bytes,5,rep,name=sql_conf,json=sqlConf,proto3\" json:\"sql_conf,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) Reset() {\n\t*x = PipelineCommand_CreateDataflowGraph{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_CreateDataflowGraph) ProtoMessage() {}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_CreateDataflowGraph.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_CreateDataflowGraph) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 0}\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) GetDefaultCatalog() string {\n\tif x != nil && x.DefaultCatalog != nil {\n\t\treturn *x.DefaultCatalog\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) GetDefaultDatabase() string {\n\tif x != nil && x.DefaultDatabase != nil {\n\t\treturn *x.DefaultDatabase\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph) GetSqlConf() map[string]string {\n\tif x != nil {\n\t\treturn x.SqlConf\n\t}\n\treturn nil\n}\n\n// Drops the graph and stops any running attached flows.\ntype PipelineCommand_DropDataflowGraph struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The graph to drop.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n}\n\nfunc (x *PipelineCommand_DropDataflowGraph) Reset() {\n\t*x = PipelineCommand_DropDataflowGraph{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_DropDataflowGraph) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_DropDataflowGraph) ProtoMessage() {}\n\nfunc (x *PipelineCommand_DropDataflowGraph) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_DropDataflowGraph.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_DropDataflowGraph) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 1}\n}\n\nfunc (x *PipelineCommand_DropDataflowGraph) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\n// Request to define a dataset: a table, a materialized view, or a temporary view.\ntype PipelineCommand_DefineDataset struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The graph to attach this dataset to.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n\t// Name of the dataset. Can be partially or fully qualified.\n\tDatasetName *string `protobuf:\"bytes,2,opt,name=dataset_name,json=datasetName,proto3,oneof\" json:\"dataset_name,omitempty\"`\n\t// The type of the dataset.\n\tDatasetType *DatasetType `protobuf:\"varint,3,opt,name=dataset_type,json=datasetType,proto3,enum=spark.connect.DatasetType,oneof\" json:\"dataset_type,omitempty\"`\n\t// Optional comment for the dataset.\n\tComment *string `protobuf:\"bytes,4,opt,name=comment,proto3,oneof\" json:\"comment,omitempty\"`\n\t// Optional table properties. Only applies to dataset_type == TABLE and dataset_type == MATERIALIZED_VIEW.\n\tTableProperties map[string]string `protobuf:\"bytes,5,rep,name=table_properties,json=tableProperties,proto3\" json:\"table_properties,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// Optional partition columns for the dataset. Only applies to dataset_type == TABLE and\n\t// dataset_type == MATERIALIZED_VIEW.\n\tPartitionCols []string `protobuf:\"bytes,6,rep,name=partition_cols,json=partitionCols,proto3\" json:\"partition_cols,omitempty\"`\n\t// Schema for the dataset. If unset, this will be inferred from incoming flows.\n\tSchema *DataType `protobuf:\"bytes,7,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n\t// The output table format of the dataset. Only applies to dataset_type == TABLE and\n\t// dataset_type == MATERIALIZED_VIEW.\n\tFormat *string `protobuf:\"bytes,8,opt,name=format,proto3,oneof\" json:\"format,omitempty\"`\n}\n\nfunc (x *PipelineCommand_DefineDataset) Reset() {\n\t*x = PipelineCommand_DefineDataset{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_DefineDataset) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_DefineDataset) ProtoMessage() {}\n\nfunc (x *PipelineCommand_DefineDataset) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_DefineDataset.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_DefineDataset) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 2}\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetDatasetName() string {\n\tif x != nil && x.DatasetName != nil {\n\t\treturn *x.DatasetName\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetDatasetType() DatasetType {\n\tif x != nil && x.DatasetType != nil {\n\t\treturn *x.DatasetType\n\t}\n\treturn DatasetType_DATASET_TYPE_UNSPECIFIED\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetComment() string {\n\tif x != nil && x.Comment != nil {\n\t\treturn *x.Comment\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetTableProperties() map[string]string {\n\tif x != nil {\n\t\treturn x.TableProperties\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetPartitionCols() []string {\n\tif x != nil {\n\t\treturn x.PartitionCols\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand_DefineDataset) GetFormat() string {\n\tif x != nil && x.Format != nil {\n\t\treturn *x.Format\n\t}\n\treturn \"\"\n}\n\n// Request to define a flow targeting a dataset.\ntype PipelineCommand_DefineFlow struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The graph to attach this flow to.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n\t// Name of the flow. For standalone flows, this must be a single-part name.\n\tFlowName *string `protobuf:\"bytes,2,opt,name=flow_name,json=flowName,proto3,oneof\" json:\"flow_name,omitempty\"`\n\t// Name of the dataset this flow writes to. Can be partially or fully qualified.\n\tTargetDatasetName *string `protobuf:\"bytes,3,opt,name=target_dataset_name,json=targetDatasetName,proto3,oneof\" json:\"target_dataset_name,omitempty\"`\n\t// An unresolved relation that defines the dataset's flow.\n\tPlan *Relation `protobuf:\"bytes,4,opt,name=plan,proto3,oneof\" json:\"plan,omitempty\"`\n\t// SQL configurations set when running this flow.\n\tSqlConf map[string]string `protobuf:\"bytes,5,rep,name=sql_conf,json=sqlConf,proto3\" json:\"sql_conf,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// If true, this flow will only be run once per full refresh.\n\tOnce *bool `protobuf:\"varint,6,opt,name=once,proto3,oneof\" json:\"once,omitempty\"`\n}\n\nfunc (x *PipelineCommand_DefineFlow) Reset() {\n\t*x = PipelineCommand_DefineFlow{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_DefineFlow) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_DefineFlow) ProtoMessage() {}\n\nfunc (x *PipelineCommand_DefineFlow) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_DefineFlow.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_DefineFlow) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 3}\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetFlowName() string {\n\tif x != nil && x.FlowName != nil {\n\t\treturn *x.FlowName\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetTargetDatasetName() string {\n\tif x != nil && x.TargetDatasetName != nil {\n\t\treturn *x.TargetDatasetName\n\t}\n\treturn \"\"\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetPlan() *Relation {\n\tif x != nil {\n\t\treturn x.Plan\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetSqlConf() map[string]string {\n\tif x != nil {\n\t\treturn x.SqlConf\n\t}\n\treturn nil\n}\n\nfunc (x *PipelineCommand_DefineFlow) GetOnce() bool {\n\tif x != nil && x.Once != nil {\n\t\treturn *x.Once\n\t}\n\treturn false\n}\n\n// Resolves all datasets and flows and start a pipeline update. Should be called after all\n// graph elements are registered.\ntype PipelineCommand_StartRun struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The graph to start.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n}\n\nfunc (x *PipelineCommand_StartRun) Reset() {\n\t*x = PipelineCommand_StartRun{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_StartRun) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_StartRun) ProtoMessage() {}\n\nfunc (x *PipelineCommand_StartRun) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_StartRun.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_StartRun) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 4}\n}\n\nfunc (x *PipelineCommand_StartRun) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\ntype PipelineCommand_CreateDataflowGraph_Response struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The ID of the created graph.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph_Response) Reset() {\n\t*x = PipelineCommand_CreateDataflowGraph_Response{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph_Response) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommand_CreateDataflowGraph_Response) ProtoMessage() {}\n\nfunc (x *PipelineCommand_CreateDataflowGraph_Response) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommand_CreateDataflowGraph_Response.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommand_CreateDataflowGraph_Response) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{0, 0, 1}\n}\n\nfunc (x *PipelineCommand_CreateDataflowGraph_Response) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\ntype PipelineCommandResult_CreateDataflowGraphResult struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// The ID of the created graph.\n\tDataflowGraphId *string `protobuf:\"bytes,1,opt,name=dataflow_graph_id,json=dataflowGraphId,proto3,oneof\" json:\"dataflow_graph_id,omitempty\"`\n}\n\nfunc (x *PipelineCommandResult_CreateDataflowGraphResult) Reset() {\n\t*x = PipelineCommandResult_CreateDataflowGraphResult{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_pipelines_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PipelineCommandResult_CreateDataflowGraphResult) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PipelineCommandResult_CreateDataflowGraphResult) ProtoMessage() {}\n\nfunc (x *PipelineCommandResult_CreateDataflowGraphResult) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_pipelines_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PipelineCommandResult_CreateDataflowGraphResult.ProtoReflect.Descriptor instead.\nfunc (*PipelineCommandResult_CreateDataflowGraphResult) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_pipelines_proto_rawDescGZIP(), []int{2, 0}\n}\n\nfunc (x *PipelineCommandResult_CreateDataflowGraphResult) GetDataflowGraphId() string {\n\tif x != nil && x.DataflowGraphId != nil {\n\t\treturn *x.DataflowGraphId\n\t}\n\treturn \"\"\n}\n\nvar File_spark_connect_pipelines_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_pipelines_proto_rawDesc = []byte{\n\t0x0a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,\n\t0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x1d,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x72, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x74, 0x79, 0x70,\n\t0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x8c, 0x11, 0x0a, 0x0f, 0x50, 0x69, 0x70,\n\t0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x68, 0x0a, 0x15,\n\t0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f,\n\t0x67, 0x72, 0x61, 0x70, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65,\n\t0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x43, 0x72, 0x65, 0x61,\n\t0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x48,\n\t0x00, 0x52, 0x13, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f,\n\t0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x55, 0x0a, 0x0e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65,\n\t0x5f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50,\n\t0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x44,\n\t0x65, 0x66, 0x69, 0x6e, 0x65, 0x44, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x48, 0x00, 0x52, 0x0d,\n\t0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x44, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x12, 0x4c, 0x0a,\n\t0x0b, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x2e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x48, 0x00, 0x52,\n\t0x0a, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x12, 0x62, 0x0a, 0x13, 0x64,\n\t0x72, 0x6f, 0x70, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61,\n\t0x70, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,\n\t0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74,\n\t0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x48, 0x00, 0x52, 0x11, 0x64, 0x72,\n\t0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12,\n\t0x46, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x72, 0x75, 0x6e, 0x18, 0x05, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x27, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6e, 0x48, 0x00, 0x52, 0x08, 0x73,\n\t0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6e, 0x12, 0x62, 0x0a, 0x19, 0x64, 0x65, 0x66, 0x69, 0x6e,\n\t0x65, 0x5f, 0x73, 0x71, 0x6c, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x65, 0x6c, 0x65, 0x6d,\n\t0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x65, 0x66, 0x69, 0x6e,\n\t0x65, 0x53, 0x71, 0x6c, 0x47, 0x72, 0x61, 0x70, 0x68, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74,\n\t0x73, 0x48, 0x00, 0x52, 0x16, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x53, 0x71, 0x6c, 0x47, 0x72,\n\t0x61, 0x70, 0x68, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x87, 0x03, 0x0a, 0x13,\n\t0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72,\n\t0x61, 0x70, 0x68, 0x12, 0x2c, 0x0a, 0x0f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63,\n\t0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0e,\n\t0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x88, 0x01,\n\t0x01, 0x12, 0x2e, 0x0a, 0x10, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x64, 0x61, 0x74,\n\t0x61, 0x62, 0x61, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0f, 0x64,\n\t0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x88, 0x01,\n\t0x01, 0x12, 0x5a, 0x0a, 0x08, 0x73, 0x71, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x18, 0x05, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x61, 0x6e, 0x64, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c,\n\t0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x2e, 0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x45,\n\t0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x73, 0x71, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x1a, 0x3a, 0x0a,\n\t0x0c, 0x53, 0x71, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a,\n\t0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12,\n\t0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x51, 0x0a, 0x08, 0x52, 0x65, 0x73,\n\t0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f,\n\t0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,\n\t0x48, 0x00, 0x52, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70,\n\t0x68, 0x49, 0x64, 0x88, 0x01, 0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66,\n\t0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x42, 0x12, 0x0a, 0x10,\n\t0x5f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67,\n\t0x42, 0x13, 0x0a, 0x11, 0x5f, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x5f, 0x64, 0x61, 0x74,\n\t0x61, 0x62, 0x61, 0x73, 0x65, 0x1a, 0x5a, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74,\n\t0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61,\n\t0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f,\n\t0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x49, 0x64, 0x88, 0x01, 0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f,\n\t0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69,\n\t0x64, 0x1a, 0xd1, 0x04, 0x0a, 0x0d, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x44, 0x61, 0x74, 0x61,\n\t0x73, 0x65, 0x74, 0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f,\n\t0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00,\n\t0x52, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x49,\n\t0x64, 0x88, 0x01, 0x01, 0x12, 0x26, 0x0a, 0x0c, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0b, 0x64, 0x61,\n\t0x74, 0x61, 0x73, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x42, 0x0a, 0x0c,\n\t0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x0e, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x48, 0x02,\n\t0x52, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01,\n\t0x12, 0x1d, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x03, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x12,\n\t0x6c, 0x0a, 0x10, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74,\n\t0x69, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69,\n\t0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x2e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65,\n\t0x44, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x50, 0x72, 0x6f,\n\t0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x74, 0x61,\n\t0x62, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x12, 0x25, 0x0a,\n\t0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6c, 0x73, 0x18,\n\t0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e,\n\t0x43, 0x6f, 0x6c, 0x73, 0x12, 0x34, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x07,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x04, 0x52,\n\t0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x66, 0x6f,\n\t0x72, 0x6d, 0x61, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x48, 0x05, 0x52, 0x06, 0x66, 0x6f,\n\t0x72, 0x6d, 0x61, 0x74, 0x88, 0x01, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x54, 0x61, 0x62, 0x6c, 0x65,\n\t0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,\n\t0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,\n\t0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f,\n\t0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69,\n\t0x64, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x42, 0x0f, 0x0a, 0x0d, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x65, 0x6e, 0x74, 0x42,\n\t0x09, 0x0a, 0x07, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x66,\n\t0x6f, 0x72, 0x6d, 0x61, 0x74, 0x1a, 0xbc, 0x03, 0x0a, 0x0a, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65,\n\t0x46, 0x6c, 0x6f, 0x77, 0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77,\n\t0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68,\n\t0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x08, 0x66, 0x6c, 0x6f, 0x77,\n\t0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x33, 0x0a, 0x13, 0x74, 0x61, 0x72, 0x67, 0x65,\n\t0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03,\n\t0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x11, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x44, 0x61,\n\t0x74, 0x61, 0x73, 0x65, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x30, 0x0a, 0x04,\n\t0x70, 0x6c, 0x61, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x48, 0x03, 0x52, 0x04, 0x70, 0x6c, 0x61, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x51,\n\t0x0a, 0x08, 0x73, 0x71, 0x6c, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x36, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64,\n\t0x2e, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x2e, 0x53, 0x71, 0x6c, 0x43,\n\t0x6f, 0x6e, 0x66, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x73, 0x71, 0x6c, 0x43, 0x6f, 0x6e,\n\t0x66, 0x12, 0x17, 0x0a, 0x04, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48,\n\t0x04, 0x52, 0x04, 0x6f, 0x6e, 0x63, 0x65, 0x88, 0x01, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x53, 0x71,\n\t0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,\n\t0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66,\n\t0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x42, 0x0c, 0x0a, 0x0a,\n\t0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x16, 0x0a, 0x14, 0x5f, 0x74,\n\t0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x73, 0x65, 0x74, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x42, 0x07, 0x0a, 0x05, 0x5f,\n\t0x6f, 0x6e, 0x63, 0x65, 0x1a, 0x51, 0x0a, 0x08, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6e,\n\t0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61,\n\t0x70, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x64,\n\t0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x49, 0x64, 0x88, 0x01,\n\t0x01, 0x42, 0x14, 0x0a, 0x12, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67,\n\t0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x42, 0x0e, 0x0a, 0x0c, 0x63, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xc7, 0x01, 0x0a, 0x16, 0x44, 0x65, 0x66, 0x69,\n\t0x6e, 0x65, 0x53, 0x71, 0x6c, 0x47, 0x72, 0x61, 0x70, 0x68, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e,\n\t0x74, 0x73, 0x12, 0x2f, 0x0a, 0x11, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67,\n\t0x72, 0x61, 0x70, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,\n\t0x0f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x49, 0x64,\n\t0x88, 0x01, 0x01, 0x12, 0x27, 0x0a, 0x0d, 0x73, 0x71, 0x6c, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f,\n\t0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0b, 0x73, 0x71,\n\t0x6c, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x74, 0x68, 0x88, 0x01, 0x01, 0x12, 0x1e, 0x0a, 0x08,\n\t0x73, 0x71, 0x6c, 0x5f, 0x74, 0x65, 0x78, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02,\n\t0x52, 0x07, 0x73, 0x71, 0x6c, 0x54, 0x65, 0x78, 0x74, 0x88, 0x01, 0x01, 0x42, 0x14, 0x0a, 0x12,\n\t0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f,\n\t0x69, 0x64, 0x42, 0x10, 0x0a, 0x0e, 0x5f, 0x73, 0x71, 0x6c, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f,\n\t0x70, 0x61, 0x74, 0x68, 0x42, 0x0b, 0x0a, 0x09, 0x5f, 0x73, 0x71, 0x6c, 0x5f, 0x74, 0x65, 0x78,\n\t0x74, 0x22, 0x8e, 0x02, 0x0a, 0x15, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x81, 0x01, 0x0a, 0x1c,\n\t0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f,\n\t0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x61,\n\t0x6e, 0x64, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44,\n\t0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x75,\n\t0x6c, 0x74, 0x48, 0x00, 0x52, 0x19, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61,\n\t0x66, 0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x1a,\n\t0x62, 0x0a, 0x19, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f,\n\t0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x2f, 0x0a, 0x11,\n\t0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68, 0x5f, 0x69,\n\t0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x66,\n\t0x6c, 0x6f, 0x77, 0x47, 0x72, 0x61, 0x70, 0x68, 0x49, 0x64, 0x88, 0x01, 0x01, 0x42, 0x14, 0x0a,\n\t0x12, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x67, 0x72, 0x61, 0x70, 0x68,\n\t0x5f, 0x69, 0x64, 0x42, 0x0d, 0x0a, 0x0b, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x74, 0x79,\n\t0x70, 0x65, 0x22, 0x49, 0x0a, 0x13, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x45, 0x76,\n\t0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x32, 0x0a, 0x05, 0x65, 0x76, 0x65,\n\t0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,\n\t0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x6b, 0x0a,\n\t0x0d, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x21,\n\t0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x88, 0x01,\n\t0x01, 0x12, 0x1d, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x09, 0x48, 0x01, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x88, 0x01, 0x01,\n\t0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x42, 0x0a,\n\t0x0a, 0x08, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2a, 0x61, 0x0a, 0x0b, 0x44, 0x61,\n\t0x74, 0x61, 0x73, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1c, 0x0a, 0x18, 0x44, 0x41, 0x54,\n\t0x41, 0x53, 0x45, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43,\n\t0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x15, 0x0a, 0x11, 0x4d, 0x41, 0x54, 0x45, 0x52,\n\t0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x01, 0x12, 0x09,\n\t0x0a, 0x05, 0x54, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x54, 0x45, 0x4d,\n\t0x50, 0x4f, 0x52, 0x41, 0x52, 0x59, 0x5f, 0x56, 0x49, 0x45, 0x57, 0x10, 0x03, 0x42, 0x36, 0x0a,\n\t0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50,\n\t0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65,\n\t0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_pipelines_proto_rawDescOnce sync.Once\n\tfile_spark_connect_pipelines_proto_rawDescData = file_spark_connect_pipelines_proto_rawDesc\n)\n\nfunc file_spark_connect_pipelines_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_pipelines_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_pipelines_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_pipelines_proto_rawDescData)\n\t})\n\treturn file_spark_connect_pipelines_proto_rawDescData\n}\n\nvar file_spark_connect_pipelines_proto_enumTypes = make([]protoimpl.EnumInfo, 1)\nvar file_spark_connect_pipelines_proto_msgTypes = make([]protoimpl.MessageInfo, 15)\nvar file_spark_connect_pipelines_proto_goTypes = []interface{}{\n\t(DatasetType)(0),                                     // 0: spark.connect.DatasetType\n\t(*PipelineCommand)(nil),                              // 1: spark.connect.PipelineCommand\n\t(*DefineSqlGraphElements)(nil),                       // 2: spark.connect.DefineSqlGraphElements\n\t(*PipelineCommandResult)(nil),                        // 3: spark.connect.PipelineCommandResult\n\t(*PipelineEventResult)(nil),                          // 4: spark.connect.PipelineEventResult\n\t(*PipelineEvent)(nil),                                // 5: spark.connect.PipelineEvent\n\t(*PipelineCommand_CreateDataflowGraph)(nil),          // 6: spark.connect.PipelineCommand.CreateDataflowGraph\n\t(*PipelineCommand_DropDataflowGraph)(nil),            // 7: spark.connect.PipelineCommand.DropDataflowGraph\n\t(*PipelineCommand_DefineDataset)(nil),                // 8: spark.connect.PipelineCommand.DefineDataset\n\t(*PipelineCommand_DefineFlow)(nil),                   // 9: spark.connect.PipelineCommand.DefineFlow\n\t(*PipelineCommand_StartRun)(nil),                     // 10: spark.connect.PipelineCommand.StartRun\n\tnil,                                                  // 11: spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntry\n\t(*PipelineCommand_CreateDataflowGraph_Response)(nil), // 12: spark.connect.PipelineCommand.CreateDataflowGraph.Response\n\tnil, // 13: spark.connect.PipelineCommand.DefineDataset.TablePropertiesEntry\n\tnil, // 14: spark.connect.PipelineCommand.DefineFlow.SqlConfEntry\n\t(*PipelineCommandResult_CreateDataflowGraphResult)(nil), // 15: spark.connect.PipelineCommandResult.CreateDataflowGraphResult\n\t(*DataType)(nil), // 16: spark.connect.DataType\n\t(*Relation)(nil), // 17: spark.connect.Relation\n}\nvar file_spark_connect_pipelines_proto_depIdxs = []int32{\n\t6,  // 0: spark.connect.PipelineCommand.create_dataflow_graph:type_name -> spark.connect.PipelineCommand.CreateDataflowGraph\n\t8,  // 1: spark.connect.PipelineCommand.define_dataset:type_name -> spark.connect.PipelineCommand.DefineDataset\n\t9,  // 2: spark.connect.PipelineCommand.define_flow:type_name -> spark.connect.PipelineCommand.DefineFlow\n\t7,  // 3: spark.connect.PipelineCommand.drop_dataflow_graph:type_name -> spark.connect.PipelineCommand.DropDataflowGraph\n\t10, // 4: spark.connect.PipelineCommand.start_run:type_name -> spark.connect.PipelineCommand.StartRun\n\t2,  // 5: spark.connect.PipelineCommand.define_sql_graph_elements:type_name -> spark.connect.DefineSqlGraphElements\n\t15, // 6: spark.connect.PipelineCommandResult.create_dataflow_graph_result:type_name -> spark.connect.PipelineCommandResult.CreateDataflowGraphResult\n\t5,  // 7: spark.connect.PipelineEventResult.event:type_name -> spark.connect.PipelineEvent\n\t11, // 8: spark.connect.PipelineCommand.CreateDataflowGraph.sql_conf:type_name -> spark.connect.PipelineCommand.CreateDataflowGraph.SqlConfEntry\n\t0,  // 9: spark.connect.PipelineCommand.DefineDataset.dataset_type:type_name -> spark.connect.DatasetType\n\t13, // 10: spark.connect.PipelineCommand.DefineDataset.table_properties:type_name -> spark.connect.PipelineCommand.DefineDataset.TablePropertiesEntry\n\t16, // 11: spark.connect.PipelineCommand.DefineDataset.schema:type_name -> spark.connect.DataType\n\t17, // 12: spark.connect.PipelineCommand.DefineFlow.plan:type_name -> spark.connect.Relation\n\t14, // 13: spark.connect.PipelineCommand.DefineFlow.sql_conf:type_name -> spark.connect.PipelineCommand.DefineFlow.SqlConfEntry\n\t14, // [14:14] is the sub-list for method output_type\n\t14, // [14:14] is the sub-list for method input_type\n\t14, // [14:14] is the sub-list for extension type_name\n\t14, // [14:14] is the sub-list for extension extendee\n\t0,  // [0:14] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_pipelines_proto_init() }\nfunc file_spark_connect_pipelines_proto_init() {\n\tif File_spark_connect_pipelines_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_relations_proto_init()\n\tfile_spark_connect_types_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_pipelines_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DefineSqlGraphElements); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommandResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineEventResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineEvent); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_CreateDataflowGraph); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_DropDataflowGraph); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_DefineDataset); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_DefineFlow); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_StartRun); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommand_CreateDataflowGraph_Response); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_pipelines_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PipelineCommandResult_CreateDataflowGraphResult); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_pipelines_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*PipelineCommand_CreateDataflowGraph_)(nil),\n\t\t(*PipelineCommand_DefineDataset_)(nil),\n\t\t(*PipelineCommand_DefineFlow_)(nil),\n\t\t(*PipelineCommand_DropDataflowGraph_)(nil),\n\t\t(*PipelineCommand_StartRun_)(nil),\n\t\t(*PipelineCommand_DefineSqlGraphElements)(nil),\n\t}\n\tfile_spark_connect_pipelines_proto_msgTypes[1].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[2].OneofWrappers = []interface{}{\n\t\t(*PipelineCommandResult_CreateDataflowGraphResult_)(nil),\n\t}\n\tfile_spark_connect_pipelines_proto_msgTypes[4].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[5].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[6].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[7].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[8].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[9].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[11].OneofWrappers = []interface{}{}\n\tfile_spark_connect_pipelines_proto_msgTypes[14].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_pipelines_proto_rawDesc,\n\t\t\tNumEnums:      1,\n\t\t\tNumMessages:   15,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_pipelines_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_pipelines_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_pipelines_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_pipelines_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_pipelines_proto = out.File\n\tfile_spark_connect_pipelines_proto_rawDesc = nil\n\tfile_spark_connect_pipelines_proto_goTypes = nil\n\tfile_spark_connect_pipelines_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/relations.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/relations.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\tanypb \"google.golang.org/protobuf/types/known/anypb\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\ntype Join_JoinType int32\n\nconst (\n\tJoin_JOIN_TYPE_UNSPECIFIED Join_JoinType = 0\n\tJoin_JOIN_TYPE_INNER       Join_JoinType = 1\n\tJoin_JOIN_TYPE_FULL_OUTER  Join_JoinType = 2\n\tJoin_JOIN_TYPE_LEFT_OUTER  Join_JoinType = 3\n\tJoin_JOIN_TYPE_RIGHT_OUTER Join_JoinType = 4\n\tJoin_JOIN_TYPE_LEFT_ANTI   Join_JoinType = 5\n\tJoin_JOIN_TYPE_LEFT_SEMI   Join_JoinType = 6\n\tJoin_JOIN_TYPE_CROSS       Join_JoinType = 7\n)\n\n// Enum value maps for Join_JoinType.\nvar (\n\tJoin_JoinType_name = map[int32]string{\n\t\t0: \"JOIN_TYPE_UNSPECIFIED\",\n\t\t1: \"JOIN_TYPE_INNER\",\n\t\t2: \"JOIN_TYPE_FULL_OUTER\",\n\t\t3: \"JOIN_TYPE_LEFT_OUTER\",\n\t\t4: \"JOIN_TYPE_RIGHT_OUTER\",\n\t\t5: \"JOIN_TYPE_LEFT_ANTI\",\n\t\t6: \"JOIN_TYPE_LEFT_SEMI\",\n\t\t7: \"JOIN_TYPE_CROSS\",\n\t}\n\tJoin_JoinType_value = map[string]int32{\n\t\t\"JOIN_TYPE_UNSPECIFIED\": 0,\n\t\t\"JOIN_TYPE_INNER\":       1,\n\t\t\"JOIN_TYPE_FULL_OUTER\":  2,\n\t\t\"JOIN_TYPE_LEFT_OUTER\":  3,\n\t\t\"JOIN_TYPE_RIGHT_OUTER\": 4,\n\t\t\"JOIN_TYPE_LEFT_ANTI\":   5,\n\t\t\"JOIN_TYPE_LEFT_SEMI\":   6,\n\t\t\"JOIN_TYPE_CROSS\":       7,\n\t}\n)\n\nfunc (x Join_JoinType) Enum() *Join_JoinType {\n\tp := new(Join_JoinType)\n\t*p = x\n\treturn p\n}\n\nfunc (x Join_JoinType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Join_JoinType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_relations_proto_enumTypes[0].Descriptor()\n}\n\nfunc (Join_JoinType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_relations_proto_enumTypes[0]\n}\n\nfunc (x Join_JoinType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Join_JoinType.Descriptor instead.\nfunc (Join_JoinType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{10, 0}\n}\n\ntype SetOperation_SetOpType int32\n\nconst (\n\tSetOperation_SET_OP_TYPE_UNSPECIFIED SetOperation_SetOpType = 0\n\tSetOperation_SET_OP_TYPE_INTERSECT   SetOperation_SetOpType = 1\n\tSetOperation_SET_OP_TYPE_UNION       SetOperation_SetOpType = 2\n\tSetOperation_SET_OP_TYPE_EXCEPT      SetOperation_SetOpType = 3\n)\n\n// Enum value maps for SetOperation_SetOpType.\nvar (\n\tSetOperation_SetOpType_name = map[int32]string{\n\t\t0: \"SET_OP_TYPE_UNSPECIFIED\",\n\t\t1: \"SET_OP_TYPE_INTERSECT\",\n\t\t2: \"SET_OP_TYPE_UNION\",\n\t\t3: \"SET_OP_TYPE_EXCEPT\",\n\t}\n\tSetOperation_SetOpType_value = map[string]int32{\n\t\t\"SET_OP_TYPE_UNSPECIFIED\": 0,\n\t\t\"SET_OP_TYPE_INTERSECT\":   1,\n\t\t\"SET_OP_TYPE_UNION\":       2,\n\t\t\"SET_OP_TYPE_EXCEPT\":      3,\n\t}\n)\n\nfunc (x SetOperation_SetOpType) Enum() *SetOperation_SetOpType {\n\tp := new(SetOperation_SetOpType)\n\t*p = x\n\treturn p\n}\n\nfunc (x SetOperation_SetOpType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (SetOperation_SetOpType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_relations_proto_enumTypes[1].Descriptor()\n}\n\nfunc (SetOperation_SetOpType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_relations_proto_enumTypes[1]\n}\n\nfunc (x SetOperation_SetOpType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use SetOperation_SetOpType.Descriptor instead.\nfunc (SetOperation_SetOpType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{11, 0}\n}\n\ntype Aggregate_GroupType int32\n\nconst (\n\tAggregate_GROUP_TYPE_UNSPECIFIED   Aggregate_GroupType = 0\n\tAggregate_GROUP_TYPE_GROUPBY       Aggregate_GroupType = 1\n\tAggregate_GROUP_TYPE_ROLLUP        Aggregate_GroupType = 2\n\tAggregate_GROUP_TYPE_CUBE          Aggregate_GroupType = 3\n\tAggregate_GROUP_TYPE_PIVOT         Aggregate_GroupType = 4\n\tAggregate_GROUP_TYPE_GROUPING_SETS Aggregate_GroupType = 5\n)\n\n// Enum value maps for Aggregate_GroupType.\nvar (\n\tAggregate_GroupType_name = map[int32]string{\n\t\t0: \"GROUP_TYPE_UNSPECIFIED\",\n\t\t1: \"GROUP_TYPE_GROUPBY\",\n\t\t2: \"GROUP_TYPE_ROLLUP\",\n\t\t3: \"GROUP_TYPE_CUBE\",\n\t\t4: \"GROUP_TYPE_PIVOT\",\n\t\t5: \"GROUP_TYPE_GROUPING_SETS\",\n\t}\n\tAggregate_GroupType_value = map[string]int32{\n\t\t\"GROUP_TYPE_UNSPECIFIED\":   0,\n\t\t\"GROUP_TYPE_GROUPBY\":       1,\n\t\t\"GROUP_TYPE_ROLLUP\":        2,\n\t\t\"GROUP_TYPE_CUBE\":          3,\n\t\t\"GROUP_TYPE_PIVOT\":         4,\n\t\t\"GROUP_TYPE_GROUPING_SETS\": 5,\n\t}\n)\n\nfunc (x Aggregate_GroupType) Enum() *Aggregate_GroupType {\n\tp := new(Aggregate_GroupType)\n\t*p = x\n\treturn p\n}\n\nfunc (x Aggregate_GroupType) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Aggregate_GroupType) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_relations_proto_enumTypes[2].Descriptor()\n}\n\nfunc (Aggregate_GroupType) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_relations_proto_enumTypes[2]\n}\n\nfunc (x Aggregate_GroupType) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Aggregate_GroupType.Descriptor instead.\nfunc (Aggregate_GroupType) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{15, 0}\n}\n\ntype Parse_ParseFormat int32\n\nconst (\n\tParse_PARSE_FORMAT_UNSPECIFIED Parse_ParseFormat = 0\n\tParse_PARSE_FORMAT_CSV         Parse_ParseFormat = 1\n\tParse_PARSE_FORMAT_JSON        Parse_ParseFormat = 2\n)\n\n// Enum value maps for Parse_ParseFormat.\nvar (\n\tParse_ParseFormat_name = map[int32]string{\n\t\t0: \"PARSE_FORMAT_UNSPECIFIED\",\n\t\t1: \"PARSE_FORMAT_CSV\",\n\t\t2: \"PARSE_FORMAT_JSON\",\n\t}\n\tParse_ParseFormat_value = map[string]int32{\n\t\t\"PARSE_FORMAT_UNSPECIFIED\": 0,\n\t\t\"PARSE_FORMAT_CSV\":         1,\n\t\t\"PARSE_FORMAT_JSON\":        2,\n\t}\n)\n\nfunc (x Parse_ParseFormat) Enum() *Parse_ParseFormat {\n\tp := new(Parse_ParseFormat)\n\t*p = x\n\treturn p\n}\n\nfunc (x Parse_ParseFormat) String() string {\n\treturn protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))\n}\n\nfunc (Parse_ParseFormat) Descriptor() protoreflect.EnumDescriptor {\n\treturn file_spark_connect_relations_proto_enumTypes[3].Descriptor()\n}\n\nfunc (Parse_ParseFormat) Type() protoreflect.EnumType {\n\treturn &file_spark_connect_relations_proto_enumTypes[3]\n}\n\nfunc (x Parse_ParseFormat) Number() protoreflect.EnumNumber {\n\treturn protoreflect.EnumNumber(x)\n}\n\n// Deprecated: Use Parse_ParseFormat.Descriptor instead.\nfunc (Parse_ParseFormat) EnumDescriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{59, 0}\n}\n\n// The main [[Relation]] type. Fundamentally, a relation is a typed container\n// that has exactly one explicit relation type set.\n//\n// When adding new relation types, they have to be registered here.\ntype Relation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tCommon *RelationCommon `protobuf:\"bytes,1,opt,name=common,proto3\" json:\"common,omitempty\"`\n\t// Types that are assignable to RelType:\n\t//\n\t//\t*Relation_Read\n\t//\t*Relation_Project\n\t//\t*Relation_Filter\n\t//\t*Relation_Join\n\t//\t*Relation_SetOp\n\t//\t*Relation_Sort\n\t//\t*Relation_Limit\n\t//\t*Relation_Aggregate\n\t//\t*Relation_Sql\n\t//\t*Relation_LocalRelation\n\t//\t*Relation_Sample\n\t//\t*Relation_Offset\n\t//\t*Relation_Deduplicate\n\t//\t*Relation_Range\n\t//\t*Relation_SubqueryAlias\n\t//\t*Relation_Repartition\n\t//\t*Relation_ToDf\n\t//\t*Relation_WithColumnsRenamed\n\t//\t*Relation_ShowString\n\t//\t*Relation_Drop\n\t//\t*Relation_Tail\n\t//\t*Relation_WithColumns\n\t//\t*Relation_Hint\n\t//\t*Relation_Unpivot\n\t//\t*Relation_ToSchema\n\t//\t*Relation_RepartitionByExpression\n\t//\t*Relation_MapPartitions\n\t//\t*Relation_CollectMetrics\n\t//\t*Relation_Parse\n\t//\t*Relation_GroupMap\n\t//\t*Relation_CoGroupMap\n\t//\t*Relation_WithWatermark\n\t//\t*Relation_ApplyInPandasWithState\n\t//\t*Relation_HtmlString\n\t//\t*Relation_CachedLocalRelation\n\t//\t*Relation_CachedRemoteRelation\n\t//\t*Relation_CommonInlineUserDefinedTableFunction\n\t//\t*Relation_AsOfJoin\n\t//\t*Relation_CommonInlineUserDefinedDataSource\n\t//\t*Relation_WithRelations\n\t//\t*Relation_Transpose\n\t//\t*Relation_UnresolvedTableValuedFunction\n\t//\t*Relation_LateralJoin\n\t//\t*Relation_FillNa\n\t//\t*Relation_DropNa\n\t//\t*Relation_Replace\n\t//\t*Relation_Summary\n\t//\t*Relation_Crosstab\n\t//\t*Relation_Describe\n\t//\t*Relation_Cov\n\t//\t*Relation_Corr\n\t//\t*Relation_ApproxQuantile\n\t//\t*Relation_FreqItems\n\t//\t*Relation_SampleBy\n\t//\t*Relation_Catalog\n\t//\t*Relation_MlRelation\n\t//\t*Relation_Extension\n\t//\t*Relation_Unknown\n\tRelType isRelation_RelType `protobuf_oneof:\"rel_type\"`\n}\n\nfunc (x *Relation) Reset() {\n\t*x = Relation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Relation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Relation) ProtoMessage() {}\n\nfunc (x *Relation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Relation.ProtoReflect.Descriptor instead.\nfunc (*Relation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (x *Relation) GetCommon() *RelationCommon {\n\tif x != nil {\n\t\treturn x.Common\n\t}\n\treturn nil\n}\n\nfunc (m *Relation) GetRelType() isRelation_RelType {\n\tif m != nil {\n\t\treturn m.RelType\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetRead() *Read {\n\tif x, ok := x.GetRelType().(*Relation_Read); ok {\n\t\treturn x.Read\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetProject() *Project {\n\tif x, ok := x.GetRelType().(*Relation_Project); ok {\n\t\treturn x.Project\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetFilter() *Filter {\n\tif x, ok := x.GetRelType().(*Relation_Filter); ok {\n\t\treturn x.Filter\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetJoin() *Join {\n\tif x, ok := x.GetRelType().(*Relation_Join); ok {\n\t\treturn x.Join\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSetOp() *SetOperation {\n\tif x, ok := x.GetRelType().(*Relation_SetOp); ok {\n\t\treturn x.SetOp\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSort() *Sort {\n\tif x, ok := x.GetRelType().(*Relation_Sort); ok {\n\t\treturn x.Sort\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetLimit() *Limit {\n\tif x, ok := x.GetRelType().(*Relation_Limit); ok {\n\t\treturn x.Limit\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetAggregate() *Aggregate {\n\tif x, ok := x.GetRelType().(*Relation_Aggregate); ok {\n\t\treturn x.Aggregate\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSql() *SQL {\n\tif x, ok := x.GetRelType().(*Relation_Sql); ok {\n\t\treturn x.Sql\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetLocalRelation() *LocalRelation {\n\tif x, ok := x.GetRelType().(*Relation_LocalRelation); ok {\n\t\treturn x.LocalRelation\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSample() *Sample {\n\tif x, ok := x.GetRelType().(*Relation_Sample); ok {\n\t\treturn x.Sample\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetOffset() *Offset {\n\tif x, ok := x.GetRelType().(*Relation_Offset); ok {\n\t\treturn x.Offset\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetDeduplicate() *Deduplicate {\n\tif x, ok := x.GetRelType().(*Relation_Deduplicate); ok {\n\t\treturn x.Deduplicate\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetRange() *Range {\n\tif x, ok := x.GetRelType().(*Relation_Range); ok {\n\t\treturn x.Range\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSubqueryAlias() *SubqueryAlias {\n\tif x, ok := x.GetRelType().(*Relation_SubqueryAlias); ok {\n\t\treturn x.SubqueryAlias\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetRepartition() *Repartition {\n\tif x, ok := x.GetRelType().(*Relation_Repartition); ok {\n\t\treturn x.Repartition\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetToDf() *ToDF {\n\tif x, ok := x.GetRelType().(*Relation_ToDf); ok {\n\t\treturn x.ToDf\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetWithColumnsRenamed() *WithColumnsRenamed {\n\tif x, ok := x.GetRelType().(*Relation_WithColumnsRenamed); ok {\n\t\treturn x.WithColumnsRenamed\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetShowString() *ShowString {\n\tif x, ok := x.GetRelType().(*Relation_ShowString); ok {\n\t\treturn x.ShowString\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetDrop() *Drop {\n\tif x, ok := x.GetRelType().(*Relation_Drop); ok {\n\t\treturn x.Drop\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetTail() *Tail {\n\tif x, ok := x.GetRelType().(*Relation_Tail); ok {\n\t\treturn x.Tail\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetWithColumns() *WithColumns {\n\tif x, ok := x.GetRelType().(*Relation_WithColumns); ok {\n\t\treturn x.WithColumns\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetHint() *Hint {\n\tif x, ok := x.GetRelType().(*Relation_Hint); ok {\n\t\treturn x.Hint\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetUnpivot() *Unpivot {\n\tif x, ok := x.GetRelType().(*Relation_Unpivot); ok {\n\t\treturn x.Unpivot\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetToSchema() *ToSchema {\n\tif x, ok := x.GetRelType().(*Relation_ToSchema); ok {\n\t\treturn x.ToSchema\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetRepartitionByExpression() *RepartitionByExpression {\n\tif x, ok := x.GetRelType().(*Relation_RepartitionByExpression); ok {\n\t\treturn x.RepartitionByExpression\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetMapPartitions() *MapPartitions {\n\tif x, ok := x.GetRelType().(*Relation_MapPartitions); ok {\n\t\treturn x.MapPartitions\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCollectMetrics() *CollectMetrics {\n\tif x, ok := x.GetRelType().(*Relation_CollectMetrics); ok {\n\t\treturn x.CollectMetrics\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetParse() *Parse {\n\tif x, ok := x.GetRelType().(*Relation_Parse); ok {\n\t\treturn x.Parse\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetGroupMap() *GroupMap {\n\tif x, ok := x.GetRelType().(*Relation_GroupMap); ok {\n\t\treturn x.GroupMap\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCoGroupMap() *CoGroupMap {\n\tif x, ok := x.GetRelType().(*Relation_CoGroupMap); ok {\n\t\treturn x.CoGroupMap\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetWithWatermark() *WithWatermark {\n\tif x, ok := x.GetRelType().(*Relation_WithWatermark); ok {\n\t\treturn x.WithWatermark\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetApplyInPandasWithState() *ApplyInPandasWithState {\n\tif x, ok := x.GetRelType().(*Relation_ApplyInPandasWithState); ok {\n\t\treturn x.ApplyInPandasWithState\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetHtmlString() *HtmlString {\n\tif x, ok := x.GetRelType().(*Relation_HtmlString); ok {\n\t\treturn x.HtmlString\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCachedLocalRelation() *CachedLocalRelation {\n\tif x, ok := x.GetRelType().(*Relation_CachedLocalRelation); ok {\n\t\treturn x.CachedLocalRelation\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCachedRemoteRelation() *CachedRemoteRelation {\n\tif x, ok := x.GetRelType().(*Relation_CachedRemoteRelation); ok {\n\t\treturn x.CachedRemoteRelation\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCommonInlineUserDefinedTableFunction() *CommonInlineUserDefinedTableFunction {\n\tif x, ok := x.GetRelType().(*Relation_CommonInlineUserDefinedTableFunction); ok {\n\t\treturn x.CommonInlineUserDefinedTableFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetAsOfJoin() *AsOfJoin {\n\tif x, ok := x.GetRelType().(*Relation_AsOfJoin); ok {\n\t\treturn x.AsOfJoin\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCommonInlineUserDefinedDataSource() *CommonInlineUserDefinedDataSource {\n\tif x, ok := x.GetRelType().(*Relation_CommonInlineUserDefinedDataSource); ok {\n\t\treturn x.CommonInlineUserDefinedDataSource\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetWithRelations() *WithRelations {\n\tif x, ok := x.GetRelType().(*Relation_WithRelations); ok {\n\t\treturn x.WithRelations\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetTranspose() *Transpose {\n\tif x, ok := x.GetRelType().(*Relation_Transpose); ok {\n\t\treturn x.Transpose\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetUnresolvedTableValuedFunction() *UnresolvedTableValuedFunction {\n\tif x, ok := x.GetRelType().(*Relation_UnresolvedTableValuedFunction); ok {\n\t\treturn x.UnresolvedTableValuedFunction\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetLateralJoin() *LateralJoin {\n\tif x, ok := x.GetRelType().(*Relation_LateralJoin); ok {\n\t\treturn x.LateralJoin\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetFillNa() *NAFill {\n\tif x, ok := x.GetRelType().(*Relation_FillNa); ok {\n\t\treturn x.FillNa\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetDropNa() *NADrop {\n\tif x, ok := x.GetRelType().(*Relation_DropNa); ok {\n\t\treturn x.DropNa\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetReplace() *NAReplace {\n\tif x, ok := x.GetRelType().(*Relation_Replace); ok {\n\t\treturn x.Replace\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSummary() *StatSummary {\n\tif x, ok := x.GetRelType().(*Relation_Summary); ok {\n\t\treturn x.Summary\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCrosstab() *StatCrosstab {\n\tif x, ok := x.GetRelType().(*Relation_Crosstab); ok {\n\t\treturn x.Crosstab\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetDescribe() *StatDescribe {\n\tif x, ok := x.GetRelType().(*Relation_Describe); ok {\n\t\treturn x.Describe\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCov() *StatCov {\n\tif x, ok := x.GetRelType().(*Relation_Cov); ok {\n\t\treturn x.Cov\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCorr() *StatCorr {\n\tif x, ok := x.GetRelType().(*Relation_Corr); ok {\n\t\treturn x.Corr\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetApproxQuantile() *StatApproxQuantile {\n\tif x, ok := x.GetRelType().(*Relation_ApproxQuantile); ok {\n\t\treturn x.ApproxQuantile\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetFreqItems() *StatFreqItems {\n\tif x, ok := x.GetRelType().(*Relation_FreqItems); ok {\n\t\treturn x.FreqItems\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetSampleBy() *StatSampleBy {\n\tif x, ok := x.GetRelType().(*Relation_SampleBy); ok {\n\t\treturn x.SampleBy\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetCatalog() *Catalog {\n\tif x, ok := x.GetRelType().(*Relation_Catalog); ok {\n\t\treturn x.Catalog\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetMlRelation() *MlRelation {\n\tif x, ok := x.GetRelType().(*Relation_MlRelation); ok {\n\t\treturn x.MlRelation\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetExtension() *anypb.Any {\n\tif x, ok := x.GetRelType().(*Relation_Extension); ok {\n\t\treturn x.Extension\n\t}\n\treturn nil\n}\n\nfunc (x *Relation) GetUnknown() *Unknown {\n\tif x, ok := x.GetRelType().(*Relation_Unknown); ok {\n\t\treturn x.Unknown\n\t}\n\treturn nil\n}\n\ntype isRelation_RelType interface {\n\tisRelation_RelType()\n}\n\ntype Relation_Read struct {\n\tRead *Read `protobuf:\"bytes,2,opt,name=read,proto3,oneof\"`\n}\n\ntype Relation_Project struct {\n\tProject *Project `protobuf:\"bytes,3,opt,name=project,proto3,oneof\"`\n}\n\ntype Relation_Filter struct {\n\tFilter *Filter `protobuf:\"bytes,4,opt,name=filter,proto3,oneof\"`\n}\n\ntype Relation_Join struct {\n\tJoin *Join `protobuf:\"bytes,5,opt,name=join,proto3,oneof\"`\n}\n\ntype Relation_SetOp struct {\n\tSetOp *SetOperation `protobuf:\"bytes,6,opt,name=set_op,json=setOp,proto3,oneof\"`\n}\n\ntype Relation_Sort struct {\n\tSort *Sort `protobuf:\"bytes,7,opt,name=sort,proto3,oneof\"`\n}\n\ntype Relation_Limit struct {\n\tLimit *Limit `protobuf:\"bytes,8,opt,name=limit,proto3,oneof\"`\n}\n\ntype Relation_Aggregate struct {\n\tAggregate *Aggregate `protobuf:\"bytes,9,opt,name=aggregate,proto3,oneof\"`\n}\n\ntype Relation_Sql struct {\n\tSql *SQL `protobuf:\"bytes,10,opt,name=sql,proto3,oneof\"`\n}\n\ntype Relation_LocalRelation struct {\n\tLocalRelation *LocalRelation `protobuf:\"bytes,11,opt,name=local_relation,json=localRelation,proto3,oneof\"`\n}\n\ntype Relation_Sample struct {\n\tSample *Sample `protobuf:\"bytes,12,opt,name=sample,proto3,oneof\"`\n}\n\ntype Relation_Offset struct {\n\tOffset *Offset `protobuf:\"bytes,13,opt,name=offset,proto3,oneof\"`\n}\n\ntype Relation_Deduplicate struct {\n\tDeduplicate *Deduplicate `protobuf:\"bytes,14,opt,name=deduplicate,proto3,oneof\"`\n}\n\ntype Relation_Range struct {\n\tRange *Range `protobuf:\"bytes,15,opt,name=range,proto3,oneof\"`\n}\n\ntype Relation_SubqueryAlias struct {\n\tSubqueryAlias *SubqueryAlias `protobuf:\"bytes,16,opt,name=subquery_alias,json=subqueryAlias,proto3,oneof\"`\n}\n\ntype Relation_Repartition struct {\n\tRepartition *Repartition `protobuf:\"bytes,17,opt,name=repartition,proto3,oneof\"`\n}\n\ntype Relation_ToDf struct {\n\tToDf *ToDF `protobuf:\"bytes,18,opt,name=to_df,json=toDf,proto3,oneof\"`\n}\n\ntype Relation_WithColumnsRenamed struct {\n\tWithColumnsRenamed *WithColumnsRenamed `protobuf:\"bytes,19,opt,name=with_columns_renamed,json=withColumnsRenamed,proto3,oneof\"`\n}\n\ntype Relation_ShowString struct {\n\tShowString *ShowString `protobuf:\"bytes,20,opt,name=show_string,json=showString,proto3,oneof\"`\n}\n\ntype Relation_Drop struct {\n\tDrop *Drop `protobuf:\"bytes,21,opt,name=drop,proto3,oneof\"`\n}\n\ntype Relation_Tail struct {\n\tTail *Tail `protobuf:\"bytes,22,opt,name=tail,proto3,oneof\"`\n}\n\ntype Relation_WithColumns struct {\n\tWithColumns *WithColumns `protobuf:\"bytes,23,opt,name=with_columns,json=withColumns,proto3,oneof\"`\n}\n\ntype Relation_Hint struct {\n\tHint *Hint `protobuf:\"bytes,24,opt,name=hint,proto3,oneof\"`\n}\n\ntype Relation_Unpivot struct {\n\tUnpivot *Unpivot `protobuf:\"bytes,25,opt,name=unpivot,proto3,oneof\"`\n}\n\ntype Relation_ToSchema struct {\n\tToSchema *ToSchema `protobuf:\"bytes,26,opt,name=to_schema,json=toSchema,proto3,oneof\"`\n}\n\ntype Relation_RepartitionByExpression struct {\n\tRepartitionByExpression *RepartitionByExpression `protobuf:\"bytes,27,opt,name=repartition_by_expression,json=repartitionByExpression,proto3,oneof\"`\n}\n\ntype Relation_MapPartitions struct {\n\tMapPartitions *MapPartitions `protobuf:\"bytes,28,opt,name=map_partitions,json=mapPartitions,proto3,oneof\"`\n}\n\ntype Relation_CollectMetrics struct {\n\tCollectMetrics *CollectMetrics `protobuf:\"bytes,29,opt,name=collect_metrics,json=collectMetrics,proto3,oneof\"`\n}\n\ntype Relation_Parse struct {\n\tParse *Parse `protobuf:\"bytes,30,opt,name=parse,proto3,oneof\"`\n}\n\ntype Relation_GroupMap struct {\n\tGroupMap *GroupMap `protobuf:\"bytes,31,opt,name=group_map,json=groupMap,proto3,oneof\"`\n}\n\ntype Relation_CoGroupMap struct {\n\tCoGroupMap *CoGroupMap `protobuf:\"bytes,32,opt,name=co_group_map,json=coGroupMap,proto3,oneof\"`\n}\n\ntype Relation_WithWatermark struct {\n\tWithWatermark *WithWatermark `protobuf:\"bytes,33,opt,name=with_watermark,json=withWatermark,proto3,oneof\"`\n}\n\ntype Relation_ApplyInPandasWithState struct {\n\tApplyInPandasWithState *ApplyInPandasWithState `protobuf:\"bytes,34,opt,name=apply_in_pandas_with_state,json=applyInPandasWithState,proto3,oneof\"`\n}\n\ntype Relation_HtmlString struct {\n\tHtmlString *HtmlString `protobuf:\"bytes,35,opt,name=html_string,json=htmlString,proto3,oneof\"`\n}\n\ntype Relation_CachedLocalRelation struct {\n\tCachedLocalRelation *CachedLocalRelation `protobuf:\"bytes,36,opt,name=cached_local_relation,json=cachedLocalRelation,proto3,oneof\"`\n}\n\ntype Relation_CachedRemoteRelation struct {\n\tCachedRemoteRelation *CachedRemoteRelation `protobuf:\"bytes,37,opt,name=cached_remote_relation,json=cachedRemoteRelation,proto3,oneof\"`\n}\n\ntype Relation_CommonInlineUserDefinedTableFunction struct {\n\tCommonInlineUserDefinedTableFunction *CommonInlineUserDefinedTableFunction `protobuf:\"bytes,38,opt,name=common_inline_user_defined_table_function,json=commonInlineUserDefinedTableFunction,proto3,oneof\"`\n}\n\ntype Relation_AsOfJoin struct {\n\tAsOfJoin *AsOfJoin `protobuf:\"bytes,39,opt,name=as_of_join,json=asOfJoin,proto3,oneof\"`\n}\n\ntype Relation_CommonInlineUserDefinedDataSource struct {\n\tCommonInlineUserDefinedDataSource *CommonInlineUserDefinedDataSource `protobuf:\"bytes,40,opt,name=common_inline_user_defined_data_source,json=commonInlineUserDefinedDataSource,proto3,oneof\"`\n}\n\ntype Relation_WithRelations struct {\n\tWithRelations *WithRelations `protobuf:\"bytes,41,opt,name=with_relations,json=withRelations,proto3,oneof\"`\n}\n\ntype Relation_Transpose struct {\n\tTranspose *Transpose `protobuf:\"bytes,42,opt,name=transpose,proto3,oneof\"`\n}\n\ntype Relation_UnresolvedTableValuedFunction struct {\n\tUnresolvedTableValuedFunction *UnresolvedTableValuedFunction `protobuf:\"bytes,43,opt,name=unresolved_table_valued_function,json=unresolvedTableValuedFunction,proto3,oneof\"`\n}\n\ntype Relation_LateralJoin struct {\n\tLateralJoin *LateralJoin `protobuf:\"bytes,44,opt,name=lateral_join,json=lateralJoin,proto3,oneof\"`\n}\n\ntype Relation_FillNa struct {\n\t// NA functions\n\tFillNa *NAFill `protobuf:\"bytes,90,opt,name=fill_na,json=fillNa,proto3,oneof\"`\n}\n\ntype Relation_DropNa struct {\n\tDropNa *NADrop `protobuf:\"bytes,91,opt,name=drop_na,json=dropNa,proto3,oneof\"`\n}\n\ntype Relation_Replace struct {\n\tReplace *NAReplace `protobuf:\"bytes,92,opt,name=replace,proto3,oneof\"`\n}\n\ntype Relation_Summary struct {\n\t// stat functions\n\tSummary *StatSummary `protobuf:\"bytes,100,opt,name=summary,proto3,oneof\"`\n}\n\ntype Relation_Crosstab struct {\n\tCrosstab *StatCrosstab `protobuf:\"bytes,101,opt,name=crosstab,proto3,oneof\"`\n}\n\ntype Relation_Describe struct {\n\tDescribe *StatDescribe `protobuf:\"bytes,102,opt,name=describe,proto3,oneof\"`\n}\n\ntype Relation_Cov struct {\n\tCov *StatCov `protobuf:\"bytes,103,opt,name=cov,proto3,oneof\"`\n}\n\ntype Relation_Corr struct {\n\tCorr *StatCorr `protobuf:\"bytes,104,opt,name=corr,proto3,oneof\"`\n}\n\ntype Relation_ApproxQuantile struct {\n\tApproxQuantile *StatApproxQuantile `protobuf:\"bytes,105,opt,name=approx_quantile,json=approxQuantile,proto3,oneof\"`\n}\n\ntype Relation_FreqItems struct {\n\tFreqItems *StatFreqItems `protobuf:\"bytes,106,opt,name=freq_items,json=freqItems,proto3,oneof\"`\n}\n\ntype Relation_SampleBy struct {\n\tSampleBy *StatSampleBy `protobuf:\"bytes,107,opt,name=sample_by,json=sampleBy,proto3,oneof\"`\n}\n\ntype Relation_Catalog struct {\n\t// Catalog API (experimental / unstable)\n\tCatalog *Catalog `protobuf:\"bytes,200,opt,name=catalog,proto3,oneof\"`\n}\n\ntype Relation_MlRelation struct {\n\t// ML relation\n\tMlRelation *MlRelation `protobuf:\"bytes,300,opt,name=ml_relation,json=mlRelation,proto3,oneof\"`\n}\n\ntype Relation_Extension struct {\n\t// This field is used to mark extensions to the protocol. When plugins generate arbitrary\n\t// relations they can add them here. During the planning the correct resolution is done.\n\tExtension *anypb.Any `protobuf:\"bytes,998,opt,name=extension,proto3,oneof\"`\n}\n\ntype Relation_Unknown struct {\n\tUnknown *Unknown `protobuf:\"bytes,999,opt,name=unknown,proto3,oneof\"`\n}\n\nfunc (*Relation_Read) isRelation_RelType() {}\n\nfunc (*Relation_Project) isRelation_RelType() {}\n\nfunc (*Relation_Filter) isRelation_RelType() {}\n\nfunc (*Relation_Join) isRelation_RelType() {}\n\nfunc (*Relation_SetOp) isRelation_RelType() {}\n\nfunc (*Relation_Sort) isRelation_RelType() {}\n\nfunc (*Relation_Limit) isRelation_RelType() {}\n\nfunc (*Relation_Aggregate) isRelation_RelType() {}\n\nfunc (*Relation_Sql) isRelation_RelType() {}\n\nfunc (*Relation_LocalRelation) isRelation_RelType() {}\n\nfunc (*Relation_Sample) isRelation_RelType() {}\n\nfunc (*Relation_Offset) isRelation_RelType() {}\n\nfunc (*Relation_Deduplicate) isRelation_RelType() {}\n\nfunc (*Relation_Range) isRelation_RelType() {}\n\nfunc (*Relation_SubqueryAlias) isRelation_RelType() {}\n\nfunc (*Relation_Repartition) isRelation_RelType() {}\n\nfunc (*Relation_ToDf) isRelation_RelType() {}\n\nfunc (*Relation_WithColumnsRenamed) isRelation_RelType() {}\n\nfunc (*Relation_ShowString) isRelation_RelType() {}\n\nfunc (*Relation_Drop) isRelation_RelType() {}\n\nfunc (*Relation_Tail) isRelation_RelType() {}\n\nfunc (*Relation_WithColumns) isRelation_RelType() {}\n\nfunc (*Relation_Hint) isRelation_RelType() {}\n\nfunc (*Relation_Unpivot) isRelation_RelType() {}\n\nfunc (*Relation_ToSchema) isRelation_RelType() {}\n\nfunc (*Relation_RepartitionByExpression) isRelation_RelType() {}\n\nfunc (*Relation_MapPartitions) isRelation_RelType() {}\n\nfunc (*Relation_CollectMetrics) isRelation_RelType() {}\n\nfunc (*Relation_Parse) isRelation_RelType() {}\n\nfunc (*Relation_GroupMap) isRelation_RelType() {}\n\nfunc (*Relation_CoGroupMap) isRelation_RelType() {}\n\nfunc (*Relation_WithWatermark) isRelation_RelType() {}\n\nfunc (*Relation_ApplyInPandasWithState) isRelation_RelType() {}\n\nfunc (*Relation_HtmlString) isRelation_RelType() {}\n\nfunc (*Relation_CachedLocalRelation) isRelation_RelType() {}\n\nfunc (*Relation_CachedRemoteRelation) isRelation_RelType() {}\n\nfunc (*Relation_CommonInlineUserDefinedTableFunction) isRelation_RelType() {}\n\nfunc (*Relation_AsOfJoin) isRelation_RelType() {}\n\nfunc (*Relation_CommonInlineUserDefinedDataSource) isRelation_RelType() {}\n\nfunc (*Relation_WithRelations) isRelation_RelType() {}\n\nfunc (*Relation_Transpose) isRelation_RelType() {}\n\nfunc (*Relation_UnresolvedTableValuedFunction) isRelation_RelType() {}\n\nfunc (*Relation_LateralJoin) isRelation_RelType() {}\n\nfunc (*Relation_FillNa) isRelation_RelType() {}\n\nfunc (*Relation_DropNa) isRelation_RelType() {}\n\nfunc (*Relation_Replace) isRelation_RelType() {}\n\nfunc (*Relation_Summary) isRelation_RelType() {}\n\nfunc (*Relation_Crosstab) isRelation_RelType() {}\n\nfunc (*Relation_Describe) isRelation_RelType() {}\n\nfunc (*Relation_Cov) isRelation_RelType() {}\n\nfunc (*Relation_Corr) isRelation_RelType() {}\n\nfunc (*Relation_ApproxQuantile) isRelation_RelType() {}\n\nfunc (*Relation_FreqItems) isRelation_RelType() {}\n\nfunc (*Relation_SampleBy) isRelation_RelType() {}\n\nfunc (*Relation_Catalog) isRelation_RelType() {}\n\nfunc (*Relation_MlRelation) isRelation_RelType() {}\n\nfunc (*Relation_Extension) isRelation_RelType() {}\n\nfunc (*Relation_Unknown) isRelation_RelType() {}\n\n// Relation to represent ML world\ntype MlRelation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to MlType:\n\t//\n\t//\t*MlRelation_Transform_\n\t//\t*MlRelation_Fetch\n\tMlType isMlRelation_MlType `protobuf_oneof:\"ml_type\"`\n}\n\nfunc (x *MlRelation) Reset() {\n\t*x = MlRelation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlRelation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlRelation) ProtoMessage() {}\n\nfunc (x *MlRelation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlRelation.ProtoReflect.Descriptor instead.\nfunc (*MlRelation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{1}\n}\n\nfunc (m *MlRelation) GetMlType() isMlRelation_MlType {\n\tif m != nil {\n\t\treturn m.MlType\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation) GetTransform() *MlRelation_Transform {\n\tif x, ok := x.GetMlType().(*MlRelation_Transform_); ok {\n\t\treturn x.Transform\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation) GetFetch() *Fetch {\n\tif x, ok := x.GetMlType().(*MlRelation_Fetch); ok {\n\t\treturn x.Fetch\n\t}\n\treturn nil\n}\n\ntype isMlRelation_MlType interface {\n\tisMlRelation_MlType()\n}\n\ntype MlRelation_Transform_ struct {\n\tTransform *MlRelation_Transform `protobuf:\"bytes,1,opt,name=transform,proto3,oneof\"`\n}\n\ntype MlRelation_Fetch struct {\n\tFetch *Fetch `protobuf:\"bytes,2,opt,name=fetch,proto3,oneof\"`\n}\n\nfunc (*MlRelation_Transform_) isMlRelation_MlType() {}\n\nfunc (*MlRelation_Fetch) isMlRelation_MlType() {}\n\n// Message for fetching attribute from object on the server side.\n// Fetch can be represented as a Relation or a ML command\n// Command: model.coefficients, model.summary.weightedPrecision which\n// returns the final literal result\n// Relation: model.summary.roc which returns a DataFrame (Relation)\ntype Fetch struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) reference to the object on the server side\n\tObjRef *ObjectRef `protobuf:\"bytes,1,opt,name=obj_ref,json=objRef,proto3\" json:\"obj_ref,omitempty\"`\n\t// (Required) the calling method chains\n\tMethods []*Fetch_Method `protobuf:\"bytes,2,rep,name=methods,proto3\" json:\"methods,omitempty\"`\n}\n\nfunc (x *Fetch) Reset() {\n\t*x = Fetch{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Fetch) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Fetch) ProtoMessage() {}\n\nfunc (x *Fetch) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Fetch.ProtoReflect.Descriptor instead.\nfunc (*Fetch) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{2}\n}\n\nfunc (x *Fetch) GetObjRef() *ObjectRef {\n\tif x != nil {\n\t\treturn x.ObjRef\n\t}\n\treturn nil\n}\n\nfunc (x *Fetch) GetMethods() []*Fetch_Method {\n\tif x != nil {\n\t\treturn x.Methods\n\t}\n\treturn nil\n}\n\n// Used for testing purposes only.\ntype Unknown struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n}\n\nfunc (x *Unknown) Reset() {\n\t*x = Unknown{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Unknown) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Unknown) ProtoMessage() {}\n\nfunc (x *Unknown) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Unknown.ProtoReflect.Descriptor instead.\nfunc (*Unknown) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{3}\n}\n\n// Common metadata of all relations.\ntype RelationCommon struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Shared relation metadata.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/relations.proto.\n\tSourceInfo string `protobuf:\"bytes,1,opt,name=source_info,json=sourceInfo,proto3\" json:\"source_info,omitempty\"`\n\t// (Optional) A per-client globally unique id for a given connect plan.\n\tPlanId *int64 `protobuf:\"varint,2,opt,name=plan_id,json=planId,proto3,oneof\" json:\"plan_id,omitempty\"`\n\t// (Optional) Keep the information of the origin for this expression such as stacktrace.\n\tOrigin *Origin `protobuf:\"bytes,3,opt,name=origin,proto3\" json:\"origin,omitempty\"`\n}\n\nfunc (x *RelationCommon) Reset() {\n\t*x = RelationCommon{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RelationCommon) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RelationCommon) ProtoMessage() {}\n\nfunc (x *RelationCommon) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RelationCommon.ProtoReflect.Descriptor instead.\nfunc (*RelationCommon) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{4}\n}\n\n// Deprecated: Marked as deprecated in spark/connect/relations.proto.\nfunc (x *RelationCommon) GetSourceInfo() string {\n\tif x != nil {\n\t\treturn x.SourceInfo\n\t}\n\treturn \"\"\n}\n\nfunc (x *RelationCommon) GetPlanId() int64 {\n\tif x != nil && x.PlanId != nil {\n\t\treturn *x.PlanId\n\t}\n\treturn 0\n}\n\nfunc (x *RelationCommon) GetOrigin() *Origin {\n\tif x != nil {\n\t\treturn x.Origin\n\t}\n\treturn nil\n}\n\n// Relation that uses a SQL query to generate the output.\ntype SQL struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The SQL query.\n\tQuery string `protobuf:\"bytes,1,opt,name=query,proto3\" json:\"query,omitempty\"`\n\t// (Optional) A map of parameter names to literal expressions.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/relations.proto.\n\tArgs map[string]*Expression_Literal `protobuf:\"bytes,2,rep,name=args,proto3\" json:\"args,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A sequence of literal expressions for positional parameters in the SQL query text.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/relations.proto.\n\tPosArgs []*Expression_Literal `protobuf:\"bytes,3,rep,name=pos_args,json=posArgs,proto3\" json:\"pos_args,omitempty\"`\n\t// (Optional) A map of parameter names to expressions.\n\t// It cannot coexist with `pos_arguments`.\n\tNamedArguments map[string]*Expression `protobuf:\"bytes,4,rep,name=named_arguments,json=namedArguments,proto3\" json:\"named_arguments,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A sequence of expressions for positional parameters in the SQL query text.\n\t// It cannot coexist with `named_arguments`.\n\tPosArguments []*Expression `protobuf:\"bytes,5,rep,name=pos_arguments,json=posArguments,proto3\" json:\"pos_arguments,omitempty\"`\n}\n\nfunc (x *SQL) Reset() {\n\t*x = SQL{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SQL) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SQL) ProtoMessage() {}\n\nfunc (x *SQL) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SQL.ProtoReflect.Descriptor instead.\nfunc (*SQL) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{5}\n}\n\nfunc (x *SQL) GetQuery() string {\n\tif x != nil {\n\t\treturn x.Query\n\t}\n\treturn \"\"\n}\n\n// Deprecated: Marked as deprecated in spark/connect/relations.proto.\nfunc (x *SQL) GetArgs() map[string]*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Args\n\t}\n\treturn nil\n}\n\n// Deprecated: Marked as deprecated in spark/connect/relations.proto.\nfunc (x *SQL) GetPosArgs() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.PosArgs\n\t}\n\treturn nil\n}\n\nfunc (x *SQL) GetNamedArguments() map[string]*Expression {\n\tif x != nil {\n\t\treturn x.NamedArguments\n\t}\n\treturn nil\n}\n\nfunc (x *SQL) GetPosArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.PosArguments\n\t}\n\treturn nil\n}\n\n// Relation of type [[WithRelations]].\n//\n// This relation contains a root plan, and one or more references that are used by the root plan.\n// There are two ways of referencing a relation, by name (through a subquery alias), or by plan_id\n// (using RelationCommon.plan_id).\n//\n// This relation can be used to implement CTEs, describe DAGs, or to reduce tree depth.\ntype WithRelations struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Plan at the root of the query tree. This plan is expected to contain one or more\n\t// references. Those references get expanded later on by the engine.\n\tRoot *Relation `protobuf:\"bytes,1,opt,name=root,proto3\" json:\"root,omitempty\"`\n\t// (Required) Plans referenced by the root plan. Relations in this list are also allowed to\n\t// contain references to other relations in this list, as long they do not form cycles.\n\tReferences []*Relation `protobuf:\"bytes,2,rep,name=references,proto3\" json:\"references,omitempty\"`\n}\n\nfunc (x *WithRelations) Reset() {\n\t*x = WithRelations{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WithRelations) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WithRelations) ProtoMessage() {}\n\nfunc (x *WithRelations) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WithRelations.ProtoReflect.Descriptor instead.\nfunc (*WithRelations) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{6}\n}\n\nfunc (x *WithRelations) GetRoot() *Relation {\n\tif x != nil {\n\t\treturn x.Root\n\t}\n\treturn nil\n}\n\nfunc (x *WithRelations) GetReferences() []*Relation {\n\tif x != nil {\n\t\treturn x.References\n\t}\n\treturn nil\n}\n\n// Relation that reads from a file / table or other data source. Does not have additional\n// inputs.\ntype Read struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ReadType:\n\t//\n\t//\t*Read_NamedTable_\n\t//\t*Read_DataSource_\n\tReadType isRead_ReadType `protobuf_oneof:\"read_type\"`\n\t// (Optional) Indicates if this is a streaming read.\n\tIsStreaming bool `protobuf:\"varint,3,opt,name=is_streaming,json=isStreaming,proto3\" json:\"is_streaming,omitempty\"`\n}\n\nfunc (x *Read) Reset() {\n\t*x = Read{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Read) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Read) ProtoMessage() {}\n\nfunc (x *Read) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Read.ProtoReflect.Descriptor instead.\nfunc (*Read) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{7}\n}\n\nfunc (m *Read) GetReadType() isRead_ReadType {\n\tif m != nil {\n\t\treturn m.ReadType\n\t}\n\treturn nil\n}\n\nfunc (x *Read) GetNamedTable() *Read_NamedTable {\n\tif x, ok := x.GetReadType().(*Read_NamedTable_); ok {\n\t\treturn x.NamedTable\n\t}\n\treturn nil\n}\n\nfunc (x *Read) GetDataSource() *Read_DataSource {\n\tif x, ok := x.GetReadType().(*Read_DataSource_); ok {\n\t\treturn x.DataSource\n\t}\n\treturn nil\n}\n\nfunc (x *Read) GetIsStreaming() bool {\n\tif x != nil {\n\t\treturn x.IsStreaming\n\t}\n\treturn false\n}\n\ntype isRead_ReadType interface {\n\tisRead_ReadType()\n}\n\ntype Read_NamedTable_ struct {\n\tNamedTable *Read_NamedTable `protobuf:\"bytes,1,opt,name=named_table,json=namedTable,proto3,oneof\"`\n}\n\ntype Read_DataSource_ struct {\n\tDataSource *Read_DataSource `protobuf:\"bytes,2,opt,name=data_source,json=dataSource,proto3,oneof\"`\n}\n\nfunc (*Read_NamedTable_) isRead_ReadType() {}\n\nfunc (*Read_DataSource_) isRead_ReadType() {}\n\n// Projection of a bag of expressions for a given input relation.\n//\n// The input relation must be specified.\n// The projected expression can be an arbitrary expression.\ntype Project struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Input relation is optional for Project.\n\t//\n\t// For example, `SELECT ABS(-1)` is valid plan without an input plan.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) A Project requires at least one expression.\n\tExpressions []*Expression `protobuf:\"bytes,3,rep,name=expressions,proto3\" json:\"expressions,omitempty\"`\n}\n\nfunc (x *Project) Reset() {\n\t*x = Project{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Project) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Project) ProtoMessage() {}\n\nfunc (x *Project) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Project.ProtoReflect.Descriptor instead.\nfunc (*Project) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{8}\n}\n\nfunc (x *Project) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Project) GetExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.Expressions\n\t}\n\treturn nil\n}\n\n// Relation that applies a boolean expression `condition` on each row of `input` to produce\n// the output result.\ntype Filter struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a Filter.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) A Filter must have a condition expression.\n\tCondition *Expression `protobuf:\"bytes,2,opt,name=condition,proto3\" json:\"condition,omitempty\"`\n}\n\nfunc (x *Filter) Reset() {\n\t*x = Filter{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Filter) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Filter) ProtoMessage() {}\n\nfunc (x *Filter) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Filter.ProtoReflect.Descriptor instead.\nfunc (*Filter) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{9}\n}\n\nfunc (x *Filter) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Filter) GetCondition() *Expression {\n\tif x != nil {\n\t\treturn x.Condition\n\t}\n\treturn nil\n}\n\n// Relation of type [[Join]].\n//\n// `left` and `right` must be present.\ntype Join struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Left input relation for a Join.\n\tLeft *Relation `protobuf:\"bytes,1,opt,name=left,proto3\" json:\"left,omitempty\"`\n\t// (Required) Right input relation for a Join.\n\tRight *Relation `protobuf:\"bytes,2,opt,name=right,proto3\" json:\"right,omitempty\"`\n\t// (Optional) The join condition. Could be unset when `using_columns` is utilized.\n\t//\n\t// This field does not co-exist with using_columns.\n\tJoinCondition *Expression `protobuf:\"bytes,3,opt,name=join_condition,json=joinCondition,proto3\" json:\"join_condition,omitempty\"`\n\t// (Required) The join type.\n\tJoinType Join_JoinType `protobuf:\"varint,4,opt,name=join_type,json=joinType,proto3,enum=spark.connect.Join_JoinType\" json:\"join_type,omitempty\"`\n\t// Optional. using_columns provides a list of columns that should present on both sides of\n\t// the join inputs that this Join will join on. For example A JOIN B USING col_name is\n\t// equivalent to A JOIN B on A.col_name = B.col_name.\n\t//\n\t// This field does not co-exist with join_condition.\n\tUsingColumns []string `protobuf:\"bytes,5,rep,name=using_columns,json=usingColumns,proto3\" json:\"using_columns,omitempty\"`\n\t// (Optional) Only used by joinWith. Set the left and right join data types.\n\tJoinDataType *Join_JoinDataType `protobuf:\"bytes,6,opt,name=join_data_type,json=joinDataType,proto3,oneof\" json:\"join_data_type,omitempty\"`\n}\n\nfunc (x *Join) Reset() {\n\t*x = Join{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Join) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Join) ProtoMessage() {}\n\nfunc (x *Join) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Join.ProtoReflect.Descriptor instead.\nfunc (*Join) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{10}\n}\n\nfunc (x *Join) GetLeft() *Relation {\n\tif x != nil {\n\t\treturn x.Left\n\t}\n\treturn nil\n}\n\nfunc (x *Join) GetRight() *Relation {\n\tif x != nil {\n\t\treturn x.Right\n\t}\n\treturn nil\n}\n\nfunc (x *Join) GetJoinCondition() *Expression {\n\tif x != nil {\n\t\treturn x.JoinCondition\n\t}\n\treturn nil\n}\n\nfunc (x *Join) GetJoinType() Join_JoinType {\n\tif x != nil {\n\t\treturn x.JoinType\n\t}\n\treturn Join_JOIN_TYPE_UNSPECIFIED\n}\n\nfunc (x *Join) GetUsingColumns() []string {\n\tif x != nil {\n\t\treturn x.UsingColumns\n\t}\n\treturn nil\n}\n\nfunc (x *Join) GetJoinDataType() *Join_JoinDataType {\n\tif x != nil {\n\t\treturn x.JoinDataType\n\t}\n\treturn nil\n}\n\n// Relation of type [[SetOperation]]\ntype SetOperation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Left input relation for a Set operation.\n\tLeftInput *Relation `protobuf:\"bytes,1,opt,name=left_input,json=leftInput,proto3\" json:\"left_input,omitempty\"`\n\t// (Required) Right input relation for a Set operation.\n\tRightInput *Relation `protobuf:\"bytes,2,opt,name=right_input,json=rightInput,proto3\" json:\"right_input,omitempty\"`\n\t// (Required) The Set operation type.\n\tSetOpType SetOperation_SetOpType `protobuf:\"varint,3,opt,name=set_op_type,json=setOpType,proto3,enum=spark.connect.SetOperation_SetOpType\" json:\"set_op_type,omitempty\"`\n\t// (Optional) If to remove duplicate rows.\n\t//\n\t// True to preserve all results.\n\t// False to remove duplicate rows.\n\tIsAll *bool `protobuf:\"varint,4,opt,name=is_all,json=isAll,proto3,oneof\" json:\"is_all,omitempty\"`\n\t// (Optional) If to perform the Set operation based on name resolution.\n\t//\n\t// Only UNION supports this option.\n\tByName *bool `protobuf:\"varint,5,opt,name=by_name,json=byName,proto3,oneof\" json:\"by_name,omitempty\"`\n\t// (Optional) If to perform the Set operation and allow missing columns.\n\t//\n\t// Only UNION supports this option.\n\tAllowMissingColumns *bool `protobuf:\"varint,6,opt,name=allow_missing_columns,json=allowMissingColumns,proto3,oneof\" json:\"allow_missing_columns,omitempty\"`\n}\n\nfunc (x *SetOperation) Reset() {\n\t*x = SetOperation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SetOperation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SetOperation) ProtoMessage() {}\n\nfunc (x *SetOperation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SetOperation.ProtoReflect.Descriptor instead.\nfunc (*SetOperation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{11}\n}\n\nfunc (x *SetOperation) GetLeftInput() *Relation {\n\tif x != nil {\n\t\treturn x.LeftInput\n\t}\n\treturn nil\n}\n\nfunc (x *SetOperation) GetRightInput() *Relation {\n\tif x != nil {\n\t\treturn x.RightInput\n\t}\n\treturn nil\n}\n\nfunc (x *SetOperation) GetSetOpType() SetOperation_SetOpType {\n\tif x != nil {\n\t\treturn x.SetOpType\n\t}\n\treturn SetOperation_SET_OP_TYPE_UNSPECIFIED\n}\n\nfunc (x *SetOperation) GetIsAll() bool {\n\tif x != nil && x.IsAll != nil {\n\t\treturn *x.IsAll\n\t}\n\treturn false\n}\n\nfunc (x *SetOperation) GetByName() bool {\n\tif x != nil && x.ByName != nil {\n\t\treturn *x.ByName\n\t}\n\treturn false\n}\n\nfunc (x *SetOperation) GetAllowMissingColumns() bool {\n\tif x != nil && x.AllowMissingColumns != nil {\n\t\treturn *x.AllowMissingColumns\n\t}\n\treturn false\n}\n\n// Relation of type [[Limit]] that is used to `limit` rows from the input relation.\ntype Limit struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a Limit.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) the limit.\n\tLimit int32 `protobuf:\"varint,2,opt,name=limit,proto3\" json:\"limit,omitempty\"`\n}\n\nfunc (x *Limit) Reset() {\n\t*x = Limit{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Limit) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Limit) ProtoMessage() {}\n\nfunc (x *Limit) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Limit.ProtoReflect.Descriptor instead.\nfunc (*Limit) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{12}\n}\n\nfunc (x *Limit) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Limit) GetLimit() int32 {\n\tif x != nil {\n\t\treturn x.Limit\n\t}\n\treturn 0\n}\n\n// Relation of type [[Offset]] that is used to read rows staring from the `offset` on\n// the input relation.\ntype Offset struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for an Offset.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) the limit.\n\tOffset int32 `protobuf:\"varint,2,opt,name=offset,proto3\" json:\"offset,omitempty\"`\n}\n\nfunc (x *Offset) Reset() {\n\t*x = Offset{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Offset) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Offset) ProtoMessage() {}\n\nfunc (x *Offset) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Offset.ProtoReflect.Descriptor instead.\nfunc (*Offset) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{13}\n}\n\nfunc (x *Offset) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Offset) GetOffset() int32 {\n\tif x != nil {\n\t\treturn x.Offset\n\t}\n\treturn 0\n}\n\n// Relation of type [[Tail]] that is used to fetch `limit` rows from the last of the input relation.\ntype Tail struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for an Tail.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) the limit.\n\tLimit int32 `protobuf:\"varint,2,opt,name=limit,proto3\" json:\"limit,omitempty\"`\n}\n\nfunc (x *Tail) Reset() {\n\t*x = Tail{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Tail) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Tail) ProtoMessage() {}\n\nfunc (x *Tail) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Tail.ProtoReflect.Descriptor instead.\nfunc (*Tail) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{14}\n}\n\nfunc (x *Tail) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Tail) GetLimit() int32 {\n\tif x != nil {\n\t\treturn x.Limit\n\t}\n\treturn 0\n}\n\n// Relation of type [[Aggregate]].\ntype Aggregate struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a RelationalGroupedDataset.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) How the RelationalGroupedDataset was built.\n\tGroupType Aggregate_GroupType `protobuf:\"varint,2,opt,name=group_type,json=groupType,proto3,enum=spark.connect.Aggregate_GroupType\" json:\"group_type,omitempty\"`\n\t// (Required) Expressions for grouping keys\n\tGroupingExpressions []*Expression `protobuf:\"bytes,3,rep,name=grouping_expressions,json=groupingExpressions,proto3\" json:\"grouping_expressions,omitempty\"`\n\t// (Required) List of values that will be translated to columns in the output DataFrame.\n\tAggregateExpressions []*Expression `protobuf:\"bytes,4,rep,name=aggregate_expressions,json=aggregateExpressions,proto3\" json:\"aggregate_expressions,omitempty\"`\n\t// (Optional) Pivots a column of the current `DataFrame` and performs the specified aggregation.\n\tPivot *Aggregate_Pivot `protobuf:\"bytes,5,opt,name=pivot,proto3\" json:\"pivot,omitempty\"`\n\t// (Optional) List of values that will be translated to columns in the output DataFrame.\n\tGroupingSets []*Aggregate_GroupingSets `protobuf:\"bytes,6,rep,name=grouping_sets,json=groupingSets,proto3\" json:\"grouping_sets,omitempty\"`\n}\n\nfunc (x *Aggregate) Reset() {\n\t*x = Aggregate{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Aggregate) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Aggregate) ProtoMessage() {}\n\nfunc (x *Aggregate) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Aggregate.ProtoReflect.Descriptor instead.\nfunc (*Aggregate) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{15}\n}\n\nfunc (x *Aggregate) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Aggregate) GetGroupType() Aggregate_GroupType {\n\tif x != nil {\n\t\treturn x.GroupType\n\t}\n\treturn Aggregate_GROUP_TYPE_UNSPECIFIED\n}\n\nfunc (x *Aggregate) GetGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.GroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *Aggregate) GetAggregateExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.AggregateExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *Aggregate) GetPivot() *Aggregate_Pivot {\n\tif x != nil {\n\t\treturn x.Pivot\n\t}\n\treturn nil\n}\n\nfunc (x *Aggregate) GetGroupingSets() []*Aggregate_GroupingSets {\n\tif x != nil {\n\t\treturn x.GroupingSets\n\t}\n\treturn nil\n}\n\n// Relation of type [[Sort]].\ntype Sort struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a Sort.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The ordering expressions\n\tOrder []*Expression_SortOrder `protobuf:\"bytes,2,rep,name=order,proto3\" json:\"order,omitempty\"`\n\t// (Optional) if this is a global sort.\n\tIsGlobal *bool `protobuf:\"varint,3,opt,name=is_global,json=isGlobal,proto3,oneof\" json:\"is_global,omitempty\"`\n}\n\nfunc (x *Sort) Reset() {\n\t*x = Sort{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Sort) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Sort) ProtoMessage() {}\n\nfunc (x *Sort) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Sort.ProtoReflect.Descriptor instead.\nfunc (*Sort) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{16}\n}\n\nfunc (x *Sort) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Sort) GetOrder() []*Expression_SortOrder {\n\tif x != nil {\n\t\treturn x.Order\n\t}\n\treturn nil\n}\n\nfunc (x *Sort) GetIsGlobal() bool {\n\tif x != nil && x.IsGlobal != nil {\n\t\treturn *x.IsGlobal\n\t}\n\treturn false\n}\n\n// Drop specified columns.\ntype Drop struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) columns to drop.\n\tColumns []*Expression `protobuf:\"bytes,2,rep,name=columns,proto3\" json:\"columns,omitempty\"`\n\t// (Optional) names of columns to drop.\n\tColumnNames []string `protobuf:\"bytes,3,rep,name=column_names,json=columnNames,proto3\" json:\"column_names,omitempty\"`\n}\n\nfunc (x *Drop) Reset() {\n\t*x = Drop{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Drop) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Drop) ProtoMessage() {}\n\nfunc (x *Drop) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Drop.ProtoReflect.Descriptor instead.\nfunc (*Drop) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{17}\n}\n\nfunc (x *Drop) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Drop) GetColumns() []*Expression {\n\tif x != nil {\n\t\treturn x.Columns\n\t}\n\treturn nil\n}\n\nfunc (x *Drop) GetColumnNames() []string {\n\tif x != nil {\n\t\treturn x.ColumnNames\n\t}\n\treturn nil\n}\n\n// Relation of type [[Deduplicate]] which have duplicate rows removed, could consider either only\n// the subset of columns or all the columns.\ntype Deduplicate struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a Deduplicate.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Deduplicate based on a list of column names.\n\t//\n\t// This field does not co-use with `all_columns_as_keys`.\n\tColumnNames []string `protobuf:\"bytes,2,rep,name=column_names,json=columnNames,proto3\" json:\"column_names,omitempty\"`\n\t// (Optional) Deduplicate based on all the columns of the input relation.\n\t//\n\t// This field does not co-use with `column_names`.\n\tAllColumnsAsKeys *bool `protobuf:\"varint,3,opt,name=all_columns_as_keys,json=allColumnsAsKeys,proto3,oneof\" json:\"all_columns_as_keys,omitempty\"`\n\t// (Optional) Deduplicate within the time range of watermark.\n\tWithinWatermark *bool `protobuf:\"varint,4,opt,name=within_watermark,json=withinWatermark,proto3,oneof\" json:\"within_watermark,omitempty\"`\n}\n\nfunc (x *Deduplicate) Reset() {\n\t*x = Deduplicate{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Deduplicate) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Deduplicate) ProtoMessage() {}\n\nfunc (x *Deduplicate) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Deduplicate.ProtoReflect.Descriptor instead.\nfunc (*Deduplicate) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{18}\n}\n\nfunc (x *Deduplicate) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Deduplicate) GetColumnNames() []string {\n\tif x != nil {\n\t\treturn x.ColumnNames\n\t}\n\treturn nil\n}\n\nfunc (x *Deduplicate) GetAllColumnsAsKeys() bool {\n\tif x != nil && x.AllColumnsAsKeys != nil {\n\t\treturn *x.AllColumnsAsKeys\n\t}\n\treturn false\n}\n\nfunc (x *Deduplicate) GetWithinWatermark() bool {\n\tif x != nil && x.WithinWatermark != nil {\n\t\treturn *x.WithinWatermark\n\t}\n\treturn false\n}\n\n// A relation that does not need to be qualified by name.\ntype LocalRelation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Local collection data serialized into Arrow IPC streaming format which contains\n\t// the schema of the data.\n\tData []byte `protobuf:\"bytes,1,opt,name=data,proto3,oneof\" json:\"data,omitempty\"`\n\t// (Optional) The schema of local data.\n\t// It should be either a DDL-formatted type string or a JSON string.\n\t//\n\t// The server side will update the column names and data types according to this schema.\n\t// If the 'data' is not provided, then this schema will be required.\n\tSchema *string `protobuf:\"bytes,2,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n}\n\nfunc (x *LocalRelation) Reset() {\n\t*x = LocalRelation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *LocalRelation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*LocalRelation) ProtoMessage() {}\n\nfunc (x *LocalRelation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use LocalRelation.ProtoReflect.Descriptor instead.\nfunc (*LocalRelation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{19}\n}\n\nfunc (x *LocalRelation) GetData() []byte {\n\tif x != nil {\n\t\treturn x.Data\n\t}\n\treturn nil\n}\n\nfunc (x *LocalRelation) GetSchema() string {\n\tif x != nil && x.Schema != nil {\n\t\treturn *x.Schema\n\t}\n\treturn \"\"\n}\n\n// A local relation that has been cached already.\ntype CachedLocalRelation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) A sha-256 hash of the serialized local relation in proto, see LocalRelation.\n\tHash string `protobuf:\"bytes,3,opt,name=hash,proto3\" json:\"hash,omitempty\"`\n}\n\nfunc (x *CachedLocalRelation) Reset() {\n\t*x = CachedLocalRelation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CachedLocalRelation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CachedLocalRelation) ProtoMessage() {}\n\nfunc (x *CachedLocalRelation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CachedLocalRelation.ProtoReflect.Descriptor instead.\nfunc (*CachedLocalRelation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{20}\n}\n\nfunc (x *CachedLocalRelation) GetHash() string {\n\tif x != nil {\n\t\treturn x.Hash\n\t}\n\treturn \"\"\n}\n\n// Represents a remote relation that has been cached on server.\ntype CachedRemoteRelation struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) ID of the remote related (assigned by the service).\n\tRelationId string `protobuf:\"bytes,1,opt,name=relation_id,json=relationId,proto3\" json:\"relation_id,omitempty\"`\n}\n\nfunc (x *CachedRemoteRelation) Reset() {\n\t*x = CachedRemoteRelation{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CachedRemoteRelation) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CachedRemoteRelation) ProtoMessage() {}\n\nfunc (x *CachedRemoteRelation) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CachedRemoteRelation.ProtoReflect.Descriptor instead.\nfunc (*CachedRemoteRelation) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{21}\n}\n\nfunc (x *CachedRemoteRelation) GetRelationId() string {\n\tif x != nil {\n\t\treturn x.RelationId\n\t}\n\treturn \"\"\n}\n\n// Relation of type [[Sample]] that samples a fraction of the dataset.\ntype Sample struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a Sample.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) lower bound.\n\tLowerBound float64 `protobuf:\"fixed64,2,opt,name=lower_bound,json=lowerBound,proto3\" json:\"lower_bound,omitempty\"`\n\t// (Required) upper bound.\n\tUpperBound float64 `protobuf:\"fixed64,3,opt,name=upper_bound,json=upperBound,proto3\" json:\"upper_bound,omitempty\"`\n\t// (Optional) Whether to sample with replacement.\n\tWithReplacement *bool `protobuf:\"varint,4,opt,name=with_replacement,json=withReplacement,proto3,oneof\" json:\"with_replacement,omitempty\"`\n\t// (Required) The random seed.\n\t// This field is required to avoid generating mutable dataframes (see SPARK-48184 for details),\n\t// however, still keep it 'optional' here for backward compatibility.\n\tSeed *int64 `protobuf:\"varint,5,opt,name=seed,proto3,oneof\" json:\"seed,omitempty\"`\n\t// (Required) Explicitly sort the underlying plan to make the ordering deterministic or cache it.\n\t// This flag is true when invoking `dataframe.randomSplit` to randomly splits DataFrame with the\n\t// provided weights. Otherwise, it is false.\n\tDeterministicOrder bool `protobuf:\"varint,6,opt,name=deterministic_order,json=deterministicOrder,proto3\" json:\"deterministic_order,omitempty\"`\n}\n\nfunc (x *Sample) Reset() {\n\t*x = Sample{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Sample) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Sample) ProtoMessage() {}\n\nfunc (x *Sample) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Sample.ProtoReflect.Descriptor instead.\nfunc (*Sample) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{22}\n}\n\nfunc (x *Sample) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Sample) GetLowerBound() float64 {\n\tif x != nil {\n\t\treturn x.LowerBound\n\t}\n\treturn 0\n}\n\nfunc (x *Sample) GetUpperBound() float64 {\n\tif x != nil {\n\t\treturn x.UpperBound\n\t}\n\treturn 0\n}\n\nfunc (x *Sample) GetWithReplacement() bool {\n\tif x != nil && x.WithReplacement != nil {\n\t\treturn *x.WithReplacement\n\t}\n\treturn false\n}\n\nfunc (x *Sample) GetSeed() int64 {\n\tif x != nil && x.Seed != nil {\n\t\treturn *x.Seed\n\t}\n\treturn 0\n}\n\nfunc (x *Sample) GetDeterministicOrder() bool {\n\tif x != nil {\n\t\treturn x.DeterministicOrder\n\t}\n\treturn false\n}\n\n// Relation of type [[Range]] that generates a sequence of integers.\ntype Range struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Default value = 0\n\tStart *int64 `protobuf:\"varint,1,opt,name=start,proto3,oneof\" json:\"start,omitempty\"`\n\t// (Required)\n\tEnd int64 `protobuf:\"varint,2,opt,name=end,proto3\" json:\"end,omitempty\"`\n\t// (Required)\n\tStep int64 `protobuf:\"varint,3,opt,name=step,proto3\" json:\"step,omitempty\"`\n\t// Optional. Default value is assigned by 1) SQL conf \"spark.sql.leafNodeDefaultParallelism\" if\n\t// it is set, or 2) spark default parallelism.\n\tNumPartitions *int32 `protobuf:\"varint,4,opt,name=num_partitions,json=numPartitions,proto3,oneof\" json:\"num_partitions,omitempty\"`\n}\n\nfunc (x *Range) Reset() {\n\t*x = Range{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Range) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Range) ProtoMessage() {}\n\nfunc (x *Range) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Range.ProtoReflect.Descriptor instead.\nfunc (*Range) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{23}\n}\n\nfunc (x *Range) GetStart() int64 {\n\tif x != nil && x.Start != nil {\n\t\treturn *x.Start\n\t}\n\treturn 0\n}\n\nfunc (x *Range) GetEnd() int64 {\n\tif x != nil {\n\t\treturn x.End\n\t}\n\treturn 0\n}\n\nfunc (x *Range) GetStep() int64 {\n\tif x != nil {\n\t\treturn x.Step\n\t}\n\treturn 0\n}\n\nfunc (x *Range) GetNumPartitions() int32 {\n\tif x != nil && x.NumPartitions != nil {\n\t\treturn *x.NumPartitions\n\t}\n\treturn 0\n}\n\n// Relation alias.\ntype SubqueryAlias struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation of SubqueryAlias.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The alias.\n\tAlias string `protobuf:\"bytes,2,opt,name=alias,proto3\" json:\"alias,omitempty\"`\n\t// (Optional) Qualifier of the alias.\n\tQualifier []string `protobuf:\"bytes,3,rep,name=qualifier,proto3\" json:\"qualifier,omitempty\"`\n}\n\nfunc (x *SubqueryAlias) Reset() {\n\t*x = SubqueryAlias{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[24]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *SubqueryAlias) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*SubqueryAlias) ProtoMessage() {}\n\nfunc (x *SubqueryAlias) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[24]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use SubqueryAlias.ProtoReflect.Descriptor instead.\nfunc (*SubqueryAlias) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{24}\n}\n\nfunc (x *SubqueryAlias) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *SubqueryAlias) GetAlias() string {\n\tif x != nil {\n\t\treturn x.Alias\n\t}\n\treturn \"\"\n}\n\nfunc (x *SubqueryAlias) GetQualifier() []string {\n\tif x != nil {\n\t\treturn x.Qualifier\n\t}\n\treturn nil\n}\n\n// Relation repartition.\ntype Repartition struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation of Repartition.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Must be positive.\n\tNumPartitions int32 `protobuf:\"varint,2,opt,name=num_partitions,json=numPartitions,proto3\" json:\"num_partitions,omitempty\"`\n\t// (Optional) Default value is false.\n\tShuffle *bool `protobuf:\"varint,3,opt,name=shuffle,proto3,oneof\" json:\"shuffle,omitempty\"`\n}\n\nfunc (x *Repartition) Reset() {\n\t*x = Repartition{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[25]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Repartition) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Repartition) ProtoMessage() {}\n\nfunc (x *Repartition) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[25]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Repartition.ProtoReflect.Descriptor instead.\nfunc (*Repartition) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{25}\n}\n\nfunc (x *Repartition) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Repartition) GetNumPartitions() int32 {\n\tif x != nil {\n\t\treturn x.NumPartitions\n\t}\n\treturn 0\n}\n\nfunc (x *Repartition) GetShuffle() bool {\n\tif x != nil && x.Shuffle != nil {\n\t\treturn *x.Shuffle\n\t}\n\treturn false\n}\n\n// Compose the string representing rows for output.\n// It will invoke 'Dataset.showString' to compute the results.\ntype ShowString struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Number of rows to show.\n\tNumRows int32 `protobuf:\"varint,2,opt,name=num_rows,json=numRows,proto3\" json:\"num_rows,omitempty\"`\n\t// (Required) If set to more than 0, truncates strings to\n\t// `truncate` characters and all cells will be aligned right.\n\tTruncate int32 `protobuf:\"varint,3,opt,name=truncate,proto3\" json:\"truncate,omitempty\"`\n\t// (Required) If set to true, prints output rows vertically (one line per column value).\n\tVertical bool `protobuf:\"varint,4,opt,name=vertical,proto3\" json:\"vertical,omitempty\"`\n}\n\nfunc (x *ShowString) Reset() {\n\t*x = ShowString{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[26]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ShowString) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ShowString) ProtoMessage() {}\n\nfunc (x *ShowString) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[26]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ShowString.ProtoReflect.Descriptor instead.\nfunc (*ShowString) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{26}\n}\n\nfunc (x *ShowString) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *ShowString) GetNumRows() int32 {\n\tif x != nil {\n\t\treturn x.NumRows\n\t}\n\treturn 0\n}\n\nfunc (x *ShowString) GetTruncate() int32 {\n\tif x != nil {\n\t\treturn x.Truncate\n\t}\n\treturn 0\n}\n\nfunc (x *ShowString) GetVertical() bool {\n\tif x != nil {\n\t\treturn x.Vertical\n\t}\n\treturn false\n}\n\n// Compose the string representing rows for output.\n// It will invoke 'Dataset.htmlString' to compute the results.\ntype HtmlString struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Number of rows to show.\n\tNumRows int32 `protobuf:\"varint,2,opt,name=num_rows,json=numRows,proto3\" json:\"num_rows,omitempty\"`\n\t// (Required) If set to more than 0, truncates strings to\n\t// `truncate` characters and all cells will be aligned right.\n\tTruncate int32 `protobuf:\"varint,3,opt,name=truncate,proto3\" json:\"truncate,omitempty\"`\n}\n\nfunc (x *HtmlString) Reset() {\n\t*x = HtmlString{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[27]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *HtmlString) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*HtmlString) ProtoMessage() {}\n\nfunc (x *HtmlString) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[27]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use HtmlString.ProtoReflect.Descriptor instead.\nfunc (*HtmlString) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{27}\n}\n\nfunc (x *HtmlString) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *HtmlString) GetNumRows() int32 {\n\tif x != nil {\n\t\treturn x.NumRows\n\t}\n\treturn 0\n}\n\nfunc (x *HtmlString) GetTruncate() int32 {\n\tif x != nil {\n\t\treturn x.Truncate\n\t}\n\treturn 0\n}\n\n// Computes specified statistics for numeric and string columns.\n// It will invoke 'Dataset.summary' (same as 'StatFunctions.summary')\n// to compute the results.\ntype StatSummary struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Statistics from to be computed.\n\t//\n\t// Available statistics are:\n\t//\n\t//\tcount\n\t//\tmean\n\t//\tstddev\n\t//\tmin\n\t//\tmax\n\t//\tarbitrary approximate percentiles specified as a percentage (e.g. 75%)\n\t//\tcount_distinct\n\t//\tapprox_count_distinct\n\t//\n\t// If no statistics are given, this function computes 'count', 'mean', 'stddev', 'min',\n\t// 'approximate quartiles' (percentiles at 25%, 50%, and 75%), and 'max'.\n\tStatistics []string `protobuf:\"bytes,2,rep,name=statistics,proto3\" json:\"statistics,omitempty\"`\n}\n\nfunc (x *StatSummary) Reset() {\n\t*x = StatSummary{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[28]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatSummary) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatSummary) ProtoMessage() {}\n\nfunc (x *StatSummary) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[28]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatSummary.ProtoReflect.Descriptor instead.\nfunc (*StatSummary) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{28}\n}\n\nfunc (x *StatSummary) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatSummary) GetStatistics() []string {\n\tif x != nil {\n\t\treturn x.Statistics\n\t}\n\treturn nil\n}\n\n// Computes basic statistics for numeric and string columns, including count, mean, stddev, min,\n// and max. If no columns are given, this function computes statistics for all numerical or\n// string columns.\ntype StatDescribe struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Columns to compute statistics on.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n}\n\nfunc (x *StatDescribe) Reset() {\n\t*x = StatDescribe{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[29]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatDescribe) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatDescribe) ProtoMessage() {}\n\nfunc (x *StatDescribe) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[29]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatDescribe.ProtoReflect.Descriptor instead.\nfunc (*StatDescribe) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{29}\n}\n\nfunc (x *StatDescribe) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatDescribe) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\n// Computes a pair-wise frequency table of the given columns. Also known as a contingency table.\n// It will invoke 'Dataset.stat.crosstab' (same as 'StatFunctions.crossTabulate')\n// to compute the results.\ntype StatCrosstab struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The name of the first column.\n\t//\n\t// Distinct items will make the first item of each row.\n\tCol1 string `protobuf:\"bytes,2,opt,name=col1,proto3\" json:\"col1,omitempty\"`\n\t// (Required) The name of the second column.\n\t//\n\t// Distinct items will make the column names of the DataFrame.\n\tCol2 string `protobuf:\"bytes,3,opt,name=col2,proto3\" json:\"col2,omitempty\"`\n}\n\nfunc (x *StatCrosstab) Reset() {\n\t*x = StatCrosstab{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[30]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatCrosstab) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatCrosstab) ProtoMessage() {}\n\nfunc (x *StatCrosstab) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[30]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatCrosstab.ProtoReflect.Descriptor instead.\nfunc (*StatCrosstab) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{30}\n}\n\nfunc (x *StatCrosstab) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatCrosstab) GetCol1() string {\n\tif x != nil {\n\t\treturn x.Col1\n\t}\n\treturn \"\"\n}\n\nfunc (x *StatCrosstab) GetCol2() string {\n\tif x != nil {\n\t\treturn x.Col2\n\t}\n\treturn \"\"\n}\n\n// Calculate the sample covariance of two numerical columns of a DataFrame.\n// It will invoke 'Dataset.stat.cov' (same as 'StatFunctions.calculateCov') to compute the results.\ntype StatCov struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The name of the first column.\n\tCol1 string `protobuf:\"bytes,2,opt,name=col1,proto3\" json:\"col1,omitempty\"`\n\t// (Required) The name of the second column.\n\tCol2 string `protobuf:\"bytes,3,opt,name=col2,proto3\" json:\"col2,omitempty\"`\n}\n\nfunc (x *StatCov) Reset() {\n\t*x = StatCov{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[31]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatCov) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatCov) ProtoMessage() {}\n\nfunc (x *StatCov) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[31]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatCov.ProtoReflect.Descriptor instead.\nfunc (*StatCov) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{31}\n}\n\nfunc (x *StatCov) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatCov) GetCol1() string {\n\tif x != nil {\n\t\treturn x.Col1\n\t}\n\treturn \"\"\n}\n\nfunc (x *StatCov) GetCol2() string {\n\tif x != nil {\n\t\treturn x.Col2\n\t}\n\treturn \"\"\n}\n\n// Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson\n// Correlation Coefficient. It will invoke 'Dataset.stat.corr' (same as\n// 'StatFunctions.pearsonCorrelation') to compute the results.\ntype StatCorr struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The name of the first column.\n\tCol1 string `protobuf:\"bytes,2,opt,name=col1,proto3\" json:\"col1,omitempty\"`\n\t// (Required) The name of the second column.\n\tCol2 string `protobuf:\"bytes,3,opt,name=col2,proto3\" json:\"col2,omitempty\"`\n\t// (Optional) Default value is 'pearson'.\n\t//\n\t// Currently only supports the Pearson Correlation Coefficient.\n\tMethod *string `protobuf:\"bytes,4,opt,name=method,proto3,oneof\" json:\"method,omitempty\"`\n}\n\nfunc (x *StatCorr) Reset() {\n\t*x = StatCorr{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[32]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatCorr) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatCorr) ProtoMessage() {}\n\nfunc (x *StatCorr) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[32]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatCorr.ProtoReflect.Descriptor instead.\nfunc (*StatCorr) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{32}\n}\n\nfunc (x *StatCorr) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatCorr) GetCol1() string {\n\tif x != nil {\n\t\treturn x.Col1\n\t}\n\treturn \"\"\n}\n\nfunc (x *StatCorr) GetCol2() string {\n\tif x != nil {\n\t\treturn x.Col2\n\t}\n\treturn \"\"\n}\n\nfunc (x *StatCorr) GetMethod() string {\n\tif x != nil && x.Method != nil {\n\t\treturn *x.Method\n\t}\n\treturn \"\"\n}\n\n// Calculates the approximate quantiles of numerical columns of a DataFrame.\n// It will invoke 'Dataset.stat.approxQuantile' (same as 'StatFunctions.approxQuantile')\n// to compute the results.\ntype StatApproxQuantile struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The names of the numerical columns.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n\t// (Required) A list of quantile probabilities.\n\t//\n\t// Each number must belong to [0, 1].\n\t// For example 0 is the minimum, 0.5 is the median, 1 is the maximum.\n\tProbabilities []float64 `protobuf:\"fixed64,3,rep,packed,name=probabilities,proto3\" json:\"probabilities,omitempty\"`\n\t// (Required) The relative target precision to achieve (greater than or equal to 0).\n\t//\n\t// If set to zero, the exact quantiles are computed, which could be very expensive.\n\t// Note that values greater than 1 are accepted but give the same result as 1.\n\tRelativeError float64 `protobuf:\"fixed64,4,opt,name=relative_error,json=relativeError,proto3\" json:\"relative_error,omitempty\"`\n}\n\nfunc (x *StatApproxQuantile) Reset() {\n\t*x = StatApproxQuantile{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[33]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatApproxQuantile) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatApproxQuantile) ProtoMessage() {}\n\nfunc (x *StatApproxQuantile) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[33]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatApproxQuantile.ProtoReflect.Descriptor instead.\nfunc (*StatApproxQuantile) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{33}\n}\n\nfunc (x *StatApproxQuantile) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatApproxQuantile) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\nfunc (x *StatApproxQuantile) GetProbabilities() []float64 {\n\tif x != nil {\n\t\treturn x.Probabilities\n\t}\n\treturn nil\n}\n\nfunc (x *StatApproxQuantile) GetRelativeError() float64 {\n\tif x != nil {\n\t\treturn x.RelativeError\n\t}\n\treturn 0\n}\n\n// Finding frequent items for columns, possibly with false positives.\n// It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems')\n// to compute the results.\ntype StatFreqItems struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The names of the columns to search frequent items in.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n\t// (Optional) The minimum frequency for an item to be considered `frequent`.\n\t// Should be greater than 1e-4.\n\tSupport *float64 `protobuf:\"fixed64,3,opt,name=support,proto3,oneof\" json:\"support,omitempty\"`\n}\n\nfunc (x *StatFreqItems) Reset() {\n\t*x = StatFreqItems{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[34]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatFreqItems) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatFreqItems) ProtoMessage() {}\n\nfunc (x *StatFreqItems) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[34]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatFreqItems.ProtoReflect.Descriptor instead.\nfunc (*StatFreqItems) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{34}\n}\n\nfunc (x *StatFreqItems) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatFreqItems) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\nfunc (x *StatFreqItems) GetSupport() float64 {\n\tif x != nil && x.Support != nil {\n\t\treturn *x.Support\n\t}\n\treturn 0\n}\n\n// Returns a stratified sample without replacement based on the fraction\n// given on each stratum.\n// It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems')\n// to compute the results.\ntype StatSampleBy struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The column that defines strata.\n\tCol *Expression `protobuf:\"bytes,2,opt,name=col,proto3\" json:\"col,omitempty\"`\n\t// (Required) Sampling fraction for each stratum.\n\t//\n\t// If a stratum is not specified, we treat its fraction as zero.\n\tFractions []*StatSampleBy_Fraction `protobuf:\"bytes,3,rep,name=fractions,proto3\" json:\"fractions,omitempty\"`\n\t// (Required) The random seed.\n\t// This field is required to avoid generating mutable dataframes (see SPARK-48184 for details),\n\t// however, still keep it 'optional' here for backward compatibility.\n\tSeed *int64 `protobuf:\"varint,5,opt,name=seed,proto3,oneof\" json:\"seed,omitempty\"`\n}\n\nfunc (x *StatSampleBy) Reset() {\n\t*x = StatSampleBy{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[35]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatSampleBy) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatSampleBy) ProtoMessage() {}\n\nfunc (x *StatSampleBy) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[35]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatSampleBy.ProtoReflect.Descriptor instead.\nfunc (*StatSampleBy) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{35}\n}\n\nfunc (x *StatSampleBy) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *StatSampleBy) GetCol() *Expression {\n\tif x != nil {\n\t\treturn x.Col\n\t}\n\treturn nil\n}\n\nfunc (x *StatSampleBy) GetFractions() []*StatSampleBy_Fraction {\n\tif x != nil {\n\t\treturn x.Fractions\n\t}\n\treturn nil\n}\n\nfunc (x *StatSampleBy) GetSeed() int64 {\n\tif x != nil && x.Seed != nil {\n\t\treturn *x.Seed\n\t}\n\treturn 0\n}\n\n// Replaces null values.\n// It will invoke 'Dataset.na.fill' (same as 'DataFrameNaFunctions.fill') to compute the results.\n// Following 3 parameter combinations are supported:\n//\n//\t1, 'values' only contains 1 item, 'cols' is empty:\n//\t  replaces null values in all type-compatible columns.\n//\t2, 'values' only contains 1 item, 'cols' is not empty:\n//\t  replaces null values in specified columns.\n//\t3, 'values' contains more than 1 items, then 'cols' is required to have the same length:\n//\t  replaces each specified column with corresponding value.\ntype NAFill struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Optional list of column names to consider.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n\t// (Required) Values to replace null values with.\n\t//\n\t// Should contain at least 1 item.\n\t// Only 4 data types are supported now: bool, long, double, string\n\tValues []*Expression_Literal `protobuf:\"bytes,3,rep,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *NAFill) Reset() {\n\t*x = NAFill{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[36]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *NAFill) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*NAFill) ProtoMessage() {}\n\nfunc (x *NAFill) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[36]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use NAFill.ProtoReflect.Descriptor instead.\nfunc (*NAFill) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{36}\n}\n\nfunc (x *NAFill) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *NAFill) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\nfunc (x *NAFill) GetValues() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\n// Drop rows containing null values.\n// It will invoke 'Dataset.na.drop' (same as 'DataFrameNaFunctions.drop') to compute the results.\ntype NADrop struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) Optional list of column names to consider.\n\t//\n\t// When it is empty, all the columns in the input relation will be considered.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n\t// (Optional) The minimum number of non-null and non-NaN values required to keep.\n\t//\n\t// When not set, it is equivalent to the number of considered columns, which means\n\t// a row will be kept only if all columns are non-null.\n\t//\n\t// 'how' options ('all', 'any') can be easily converted to this field:\n\t//   - 'all' -> set 'min_non_nulls' 1;\n\t//   - 'any' -> keep 'min_non_nulls' unset;\n\tMinNonNulls *int32 `protobuf:\"varint,3,opt,name=min_non_nulls,json=minNonNulls,proto3,oneof\" json:\"min_non_nulls,omitempty\"`\n}\n\nfunc (x *NADrop) Reset() {\n\t*x = NADrop{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[37]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *NADrop) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*NADrop) ProtoMessage() {}\n\nfunc (x *NADrop) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[37]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use NADrop.ProtoReflect.Descriptor instead.\nfunc (*NADrop) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{37}\n}\n\nfunc (x *NADrop) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *NADrop) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\nfunc (x *NADrop) GetMinNonNulls() int32 {\n\tif x != nil && x.MinNonNulls != nil {\n\t\treturn *x.MinNonNulls\n\t}\n\treturn 0\n}\n\n// Replaces old values with the corresponding values.\n// It will invoke 'Dataset.na.replace' (same as 'DataFrameNaFunctions.replace')\n// to compute the results.\ntype NAReplace struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) List of column names to consider.\n\t//\n\t// When it is empty, all the type-compatible columns in the input relation will be considered.\n\tCols []string `protobuf:\"bytes,2,rep,name=cols,proto3\" json:\"cols,omitempty\"`\n\t// (Optional) The value replacement mapping.\n\tReplacements []*NAReplace_Replacement `protobuf:\"bytes,3,rep,name=replacements,proto3\" json:\"replacements,omitempty\"`\n}\n\nfunc (x *NAReplace) Reset() {\n\t*x = NAReplace{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[38]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *NAReplace) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*NAReplace) ProtoMessage() {}\n\nfunc (x *NAReplace) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[38]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use NAReplace.ProtoReflect.Descriptor instead.\nfunc (*NAReplace) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{38}\n}\n\nfunc (x *NAReplace) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *NAReplace) GetCols() []string {\n\tif x != nil {\n\t\treturn x.Cols\n\t}\n\treturn nil\n}\n\nfunc (x *NAReplace) GetReplacements() []*NAReplace_Replacement {\n\tif x != nil {\n\t\treturn x.Replacements\n\t}\n\treturn nil\n}\n\n// Rename columns on the input relation by the same length of names.\ntype ToDF struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation of RenameColumnsBySameLengthNames.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required)\n\t//\n\t// The number of columns of the input relation must be equal to the length\n\t// of this field. If this is not true, an exception will be returned.\n\tColumnNames []string `protobuf:\"bytes,2,rep,name=column_names,json=columnNames,proto3\" json:\"column_names,omitempty\"`\n}\n\nfunc (x *ToDF) Reset() {\n\t*x = ToDF{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[39]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ToDF) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ToDF) ProtoMessage() {}\n\nfunc (x *ToDF) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[39]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ToDF.ProtoReflect.Descriptor instead.\nfunc (*ToDF) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{39}\n}\n\nfunc (x *ToDF) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *ToDF) GetColumnNames() []string {\n\tif x != nil {\n\t\treturn x.ColumnNames\n\t}\n\treturn nil\n}\n\n// Rename columns on the input relation by a map with name to name mapping.\ntype WithColumnsRenamed struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional)\n\t//\n\t// Renaming column names of input relation from A to B where A is the map key\n\t// and B is the map value. This is a no-op if schema doesn't contain any A. It\n\t// does not require that all input relation column names to present as keys.\n\t// duplicated B are not allowed.\n\t//\n\t// Deprecated: Marked as deprecated in spark/connect/relations.proto.\n\tRenameColumnsMap map[string]string            `protobuf:\"bytes,2,rep,name=rename_columns_map,json=renameColumnsMap,proto3\" json:\"rename_columns_map,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\tRenames          []*WithColumnsRenamed_Rename `protobuf:\"bytes,3,rep,name=renames,proto3\" json:\"renames,omitempty\"`\n}\n\nfunc (x *WithColumnsRenamed) Reset() {\n\t*x = WithColumnsRenamed{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[40]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WithColumnsRenamed) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WithColumnsRenamed) ProtoMessage() {}\n\nfunc (x *WithColumnsRenamed) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[40]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WithColumnsRenamed.ProtoReflect.Descriptor instead.\nfunc (*WithColumnsRenamed) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{40}\n}\n\nfunc (x *WithColumnsRenamed) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\n// Deprecated: Marked as deprecated in spark/connect/relations.proto.\nfunc (x *WithColumnsRenamed) GetRenameColumnsMap() map[string]string {\n\tif x != nil {\n\t\treturn x.RenameColumnsMap\n\t}\n\treturn nil\n}\n\nfunc (x *WithColumnsRenamed) GetRenames() []*WithColumnsRenamed_Rename {\n\tif x != nil {\n\t\treturn x.Renames\n\t}\n\treturn nil\n}\n\n// Adding columns or replacing the existing columns that have the same names.\ntype WithColumns struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required)\n\t//\n\t// Given a column name, apply the corresponding expression on the column. If column\n\t// name exists in the input relation, then replace the column. If the column name\n\t// does not exist in the input relation, then adds it as a new column.\n\t//\n\t// Only one name part is expected from each Expression.Alias.\n\t//\n\t// An exception is thrown when duplicated names are present in the mapping.\n\tAliases []*Expression_Alias `protobuf:\"bytes,2,rep,name=aliases,proto3\" json:\"aliases,omitempty\"`\n}\n\nfunc (x *WithColumns) Reset() {\n\t*x = WithColumns{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[41]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WithColumns) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WithColumns) ProtoMessage() {}\n\nfunc (x *WithColumns) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[41]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WithColumns.ProtoReflect.Descriptor instead.\nfunc (*WithColumns) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{41}\n}\n\nfunc (x *WithColumns) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *WithColumns) GetAliases() []*Expression_Alias {\n\tif x != nil {\n\t\treturn x.Aliases\n\t}\n\treturn nil\n}\n\ntype WithWatermark struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Name of the column containing event time.\n\tEventTime string `protobuf:\"bytes,2,opt,name=event_time,json=eventTime,proto3\" json:\"event_time,omitempty\"`\n\t// (Required)\n\tDelayThreshold string `protobuf:\"bytes,3,opt,name=delay_threshold,json=delayThreshold,proto3\" json:\"delay_threshold,omitempty\"`\n}\n\nfunc (x *WithWatermark) Reset() {\n\t*x = WithWatermark{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[42]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WithWatermark) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WithWatermark) ProtoMessage() {}\n\nfunc (x *WithWatermark) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[42]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WithWatermark.ProtoReflect.Descriptor instead.\nfunc (*WithWatermark) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{42}\n}\n\nfunc (x *WithWatermark) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *WithWatermark) GetEventTime() string {\n\tif x != nil {\n\t\treturn x.EventTime\n\t}\n\treturn \"\"\n}\n\nfunc (x *WithWatermark) GetDelayThreshold() string {\n\tif x != nil {\n\t\treturn x.DelayThreshold\n\t}\n\treturn \"\"\n}\n\n// Specify a hint over a relation. Hint should have a name and optional parameters.\ntype Hint struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Hint name.\n\t//\n\t// Supported Join hints include BROADCAST, MERGE, SHUFFLE_HASH, SHUFFLE_REPLICATE_NL.\n\t//\n\t// Supported partitioning hints include COALESCE, REPARTITION, REPARTITION_BY_RANGE.\n\tName string `protobuf:\"bytes,2,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Optional) Hint parameters.\n\tParameters []*Expression `protobuf:\"bytes,3,rep,name=parameters,proto3\" json:\"parameters,omitempty\"`\n}\n\nfunc (x *Hint) Reset() {\n\t*x = Hint{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[43]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Hint) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Hint) ProtoMessage() {}\n\nfunc (x *Hint) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[43]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Hint.ProtoReflect.Descriptor instead.\nfunc (*Hint) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{43}\n}\n\nfunc (x *Hint) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Hint) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *Hint) GetParameters() []*Expression {\n\tif x != nil {\n\t\treturn x.Parameters\n\t}\n\treturn nil\n}\n\n// Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set.\ntype Unpivot struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Id columns.\n\tIds []*Expression `protobuf:\"bytes,2,rep,name=ids,proto3\" json:\"ids,omitempty\"`\n\t// (Optional) Value columns to unpivot.\n\tValues *Unpivot_Values `protobuf:\"bytes,3,opt,name=values,proto3,oneof\" json:\"values,omitempty\"`\n\t// (Required) Name of the variable column.\n\tVariableColumnName string `protobuf:\"bytes,4,opt,name=variable_column_name,json=variableColumnName,proto3\" json:\"variable_column_name,omitempty\"`\n\t// (Required) Name of the value column.\n\tValueColumnName string `protobuf:\"bytes,5,opt,name=value_column_name,json=valueColumnName,proto3\" json:\"value_column_name,omitempty\"`\n}\n\nfunc (x *Unpivot) Reset() {\n\t*x = Unpivot{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[44]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Unpivot) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Unpivot) ProtoMessage() {}\n\nfunc (x *Unpivot) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[44]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Unpivot.ProtoReflect.Descriptor instead.\nfunc (*Unpivot) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{44}\n}\n\nfunc (x *Unpivot) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Unpivot) GetIds() []*Expression {\n\tif x != nil {\n\t\treturn x.Ids\n\t}\n\treturn nil\n}\n\nfunc (x *Unpivot) GetValues() *Unpivot_Values {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\nfunc (x *Unpivot) GetVariableColumnName() string {\n\tif x != nil {\n\t\treturn x.VariableColumnName\n\t}\n\treturn \"\"\n}\n\nfunc (x *Unpivot) GetValueColumnName() string {\n\tif x != nil {\n\t\treturn x.ValueColumnName\n\t}\n\treturn \"\"\n}\n\n// Transpose a DataFrame, switching rows to columns.\n// Transforms the DataFrame such that the values in the specified index column\n// become the new columns of the DataFrame.\ntype Transpose struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Optional) A list of columns that will be treated as the indices.\n\t// Only single column is supported now.\n\tIndexColumns []*Expression `protobuf:\"bytes,2,rep,name=index_columns,json=indexColumns,proto3\" json:\"index_columns,omitempty\"`\n}\n\nfunc (x *Transpose) Reset() {\n\t*x = Transpose{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[45]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Transpose) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Transpose) ProtoMessage() {}\n\nfunc (x *Transpose) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[45]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Transpose.ProtoReflect.Descriptor instead.\nfunc (*Transpose) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{45}\n}\n\nfunc (x *Transpose) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Transpose) GetIndexColumns() []*Expression {\n\tif x != nil {\n\t\treturn x.IndexColumns\n\t}\n\treturn nil\n}\n\ntype UnresolvedTableValuedFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) name (or unparsed name for user defined function) for the unresolved function.\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional) Function arguments. Empty arguments are allowed.\n\tArguments []*Expression `protobuf:\"bytes,2,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n}\n\nfunc (x *UnresolvedTableValuedFunction) Reset() {\n\t*x = UnresolvedTableValuedFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[46]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *UnresolvedTableValuedFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*UnresolvedTableValuedFunction) ProtoMessage() {}\n\nfunc (x *UnresolvedTableValuedFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[46]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use UnresolvedTableValuedFunction.ProtoReflect.Descriptor instead.\nfunc (*UnresolvedTableValuedFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{46}\n}\n\nfunc (x *UnresolvedTableValuedFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *UnresolvedTableValuedFunction) GetArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\ntype ToSchema struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The user provided schema.\n\t//\n\t// The Sever side will update the dataframe with this schema.\n\tSchema *DataType `protobuf:\"bytes,2,opt,name=schema,proto3\" json:\"schema,omitempty\"`\n}\n\nfunc (x *ToSchema) Reset() {\n\t*x = ToSchema{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[47]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ToSchema) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ToSchema) ProtoMessage() {}\n\nfunc (x *ToSchema) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[47]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ToSchema.ProtoReflect.Descriptor instead.\nfunc (*ToSchema) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{47}\n}\n\nfunc (x *ToSchema) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *ToSchema) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\ntype RepartitionByExpression struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The partitioning expressions.\n\tPartitionExprs []*Expression `protobuf:\"bytes,2,rep,name=partition_exprs,json=partitionExprs,proto3\" json:\"partition_exprs,omitempty\"`\n\t// (Optional) number of partitions, must be positive.\n\tNumPartitions *int32 `protobuf:\"varint,3,opt,name=num_partitions,json=numPartitions,proto3,oneof\" json:\"num_partitions,omitempty\"`\n}\n\nfunc (x *RepartitionByExpression) Reset() {\n\t*x = RepartitionByExpression{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[48]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *RepartitionByExpression) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*RepartitionByExpression) ProtoMessage() {}\n\nfunc (x *RepartitionByExpression) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[48]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use RepartitionByExpression.ProtoReflect.Descriptor instead.\nfunc (*RepartitionByExpression) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{48}\n}\n\nfunc (x *RepartitionByExpression) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *RepartitionByExpression) GetPartitionExprs() []*Expression {\n\tif x != nil {\n\t\treturn x.PartitionExprs\n\t}\n\treturn nil\n}\n\nfunc (x *RepartitionByExpression) GetNumPartitions() int32 {\n\tif x != nil && x.NumPartitions != nil {\n\t\treturn *x.NumPartitions\n\t}\n\treturn 0\n}\n\ntype MapPartitions struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for a mapPartitions-equivalent API: mapInPandas, mapInArrow.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Input user-defined function.\n\tFunc *CommonInlineUserDefinedFunction `protobuf:\"bytes,2,opt,name=func,proto3\" json:\"func,omitempty\"`\n\t// (Optional) Whether to use barrier mode execution or not.\n\tIsBarrier *bool `protobuf:\"varint,3,opt,name=is_barrier,json=isBarrier,proto3,oneof\" json:\"is_barrier,omitempty\"`\n\t// (Optional) ResourceProfile id used for the stage level scheduling.\n\tProfileId *int32 `protobuf:\"varint,4,opt,name=profile_id,json=profileId,proto3,oneof\" json:\"profile_id,omitempty\"`\n}\n\nfunc (x *MapPartitions) Reset() {\n\t*x = MapPartitions{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[49]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MapPartitions) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MapPartitions) ProtoMessage() {}\n\nfunc (x *MapPartitions) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[49]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MapPartitions.ProtoReflect.Descriptor instead.\nfunc (*MapPartitions) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{49}\n}\n\nfunc (x *MapPartitions) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *MapPartitions) GetFunc() *CommonInlineUserDefinedFunction {\n\tif x != nil {\n\t\treturn x.Func\n\t}\n\treturn nil\n}\n\nfunc (x *MapPartitions) GetIsBarrier() bool {\n\tif x != nil && x.IsBarrier != nil {\n\t\treturn *x.IsBarrier\n\t}\n\treturn false\n}\n\nfunc (x *MapPartitions) GetProfileId() int32 {\n\tif x != nil && x.ProfileId != nil {\n\t\treturn *x.ProfileId\n\t}\n\treturn 0\n}\n\ntype GroupMap struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for Group Map API: apply, applyInPandas.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Expressions for grouping keys.\n\tGroupingExpressions []*Expression `protobuf:\"bytes,2,rep,name=grouping_expressions,json=groupingExpressions,proto3\" json:\"grouping_expressions,omitempty\"`\n\t// (Required) Input user-defined function.\n\tFunc *CommonInlineUserDefinedFunction `protobuf:\"bytes,3,opt,name=func,proto3\" json:\"func,omitempty\"`\n\t// (Optional) Expressions for sorting. Only used by Scala Sorted Group Map API.\n\tSortingExpressions []*Expression `protobuf:\"bytes,4,rep,name=sorting_expressions,json=sortingExpressions,proto3\" json:\"sorting_expressions,omitempty\"`\n\t// Below fields are only used by (Flat)MapGroupsWithState\n\t// (Optional) Input relation for initial State.\n\tInitialInput *Relation `protobuf:\"bytes,5,opt,name=initial_input,json=initialInput,proto3\" json:\"initial_input,omitempty\"`\n\t// (Optional) Expressions for grouping keys of the initial state input relation.\n\tInitialGroupingExpressions []*Expression `protobuf:\"bytes,6,rep,name=initial_grouping_expressions,json=initialGroupingExpressions,proto3\" json:\"initial_grouping_expressions,omitempty\"`\n\t// (Optional) True if MapGroupsWithState, false if FlatMapGroupsWithState.\n\tIsMapGroupsWithState *bool `protobuf:\"varint,7,opt,name=is_map_groups_with_state,json=isMapGroupsWithState,proto3,oneof\" json:\"is_map_groups_with_state,omitempty\"`\n\t// (Optional) The output mode of the function.\n\tOutputMode *string `protobuf:\"bytes,8,opt,name=output_mode,json=outputMode,proto3,oneof\" json:\"output_mode,omitempty\"`\n\t// (Optional) Timeout configuration for groups that do not receive data for a while.\n\tTimeoutConf *string `protobuf:\"bytes,9,opt,name=timeout_conf,json=timeoutConf,proto3,oneof\" json:\"timeout_conf,omitempty\"`\n\t// (Optional) The schema for the grouped state.\n\tStateSchema *DataType `protobuf:\"bytes,10,opt,name=state_schema,json=stateSchema,proto3,oneof\" json:\"state_schema,omitempty\"`\n\t// Below fields are used by TransformWithState and TransformWithStateInPandas\n\t// (Optional) TransformWithState related parameters.\n\tTransformWithStateInfo *TransformWithStateInfo `protobuf:\"bytes,11,opt,name=transform_with_state_info,json=transformWithStateInfo,proto3,oneof\" json:\"transform_with_state_info,omitempty\"`\n}\n\nfunc (x *GroupMap) Reset() {\n\t*x = GroupMap{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[50]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *GroupMap) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*GroupMap) ProtoMessage() {}\n\nfunc (x *GroupMap) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[50]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use GroupMap.ProtoReflect.Descriptor instead.\nfunc (*GroupMap) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{50}\n}\n\nfunc (x *GroupMap) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.GroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetFunc() *CommonInlineUserDefinedFunction {\n\tif x != nil {\n\t\treturn x.Func\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetSortingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.SortingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetInitialInput() *Relation {\n\tif x != nil {\n\t\treturn x.InitialInput\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetInitialGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.InitialGroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetIsMapGroupsWithState() bool {\n\tif x != nil && x.IsMapGroupsWithState != nil {\n\t\treturn *x.IsMapGroupsWithState\n\t}\n\treturn false\n}\n\nfunc (x *GroupMap) GetOutputMode() string {\n\tif x != nil && x.OutputMode != nil {\n\t\treturn *x.OutputMode\n\t}\n\treturn \"\"\n}\n\nfunc (x *GroupMap) GetTimeoutConf() string {\n\tif x != nil && x.TimeoutConf != nil {\n\t\treturn *x.TimeoutConf\n\t}\n\treturn \"\"\n}\n\nfunc (x *GroupMap) GetStateSchema() *DataType {\n\tif x != nil {\n\t\treturn x.StateSchema\n\t}\n\treturn nil\n}\n\nfunc (x *GroupMap) GetTransformWithStateInfo() *TransformWithStateInfo {\n\tif x != nil {\n\t\treturn x.TransformWithStateInfo\n\t}\n\treturn nil\n}\n\n// Additional input parameters used for TransformWithState operator.\ntype TransformWithStateInfo struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Time mode string for transformWithState.\n\tTimeMode string `protobuf:\"bytes,1,opt,name=time_mode,json=timeMode,proto3\" json:\"time_mode,omitempty\"`\n\t// (Optional) Event time column name.\n\tEventTimeColumnName *string `protobuf:\"bytes,2,opt,name=event_time_column_name,json=eventTimeColumnName,proto3,oneof\" json:\"event_time_column_name,omitempty\"`\n\t// (Optional) Schema for the output DataFrame.\n\t// Only required used for TransformWithStateInPandas.\n\tOutputSchema *DataType `protobuf:\"bytes,3,opt,name=output_schema,json=outputSchema,proto3,oneof\" json:\"output_schema,omitempty\"`\n}\n\nfunc (x *TransformWithStateInfo) Reset() {\n\t*x = TransformWithStateInfo{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[51]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *TransformWithStateInfo) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*TransformWithStateInfo) ProtoMessage() {}\n\nfunc (x *TransformWithStateInfo) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[51]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use TransformWithStateInfo.ProtoReflect.Descriptor instead.\nfunc (*TransformWithStateInfo) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{51}\n}\n\nfunc (x *TransformWithStateInfo) GetTimeMode() string {\n\tif x != nil {\n\t\treturn x.TimeMode\n\t}\n\treturn \"\"\n}\n\nfunc (x *TransformWithStateInfo) GetEventTimeColumnName() string {\n\tif x != nil && x.EventTimeColumnName != nil {\n\t\treturn *x.EventTimeColumnName\n\t}\n\treturn \"\"\n}\n\nfunc (x *TransformWithStateInfo) GetOutputSchema() *DataType {\n\tif x != nil {\n\t\treturn x.OutputSchema\n\t}\n\treturn nil\n}\n\ntype CoGroupMap struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) One input relation for CoGroup Map API - applyInPandas.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// Expressions for grouping keys of the first input relation.\n\tInputGroupingExpressions []*Expression `protobuf:\"bytes,2,rep,name=input_grouping_expressions,json=inputGroupingExpressions,proto3\" json:\"input_grouping_expressions,omitempty\"`\n\t// (Required) The other input relation.\n\tOther *Relation `protobuf:\"bytes,3,opt,name=other,proto3\" json:\"other,omitempty\"`\n\t// Expressions for grouping keys of the other input relation.\n\tOtherGroupingExpressions []*Expression `protobuf:\"bytes,4,rep,name=other_grouping_expressions,json=otherGroupingExpressions,proto3\" json:\"other_grouping_expressions,omitempty\"`\n\t// (Required) Input user-defined function.\n\tFunc *CommonInlineUserDefinedFunction `protobuf:\"bytes,5,opt,name=func,proto3\" json:\"func,omitempty\"`\n\t// (Optional) Expressions for sorting. Only used by Scala Sorted CoGroup Map API.\n\tInputSortingExpressions []*Expression `protobuf:\"bytes,6,rep,name=input_sorting_expressions,json=inputSortingExpressions,proto3\" json:\"input_sorting_expressions,omitempty\"`\n\t// (Optional) Expressions for sorting. Only used by Scala Sorted CoGroup Map API.\n\tOtherSortingExpressions []*Expression `protobuf:\"bytes,7,rep,name=other_sorting_expressions,json=otherSortingExpressions,proto3\" json:\"other_sorting_expressions,omitempty\"`\n}\n\nfunc (x *CoGroupMap) Reset() {\n\t*x = CoGroupMap{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[52]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CoGroupMap) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CoGroupMap) ProtoMessage() {}\n\nfunc (x *CoGroupMap) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[52]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CoGroupMap.ProtoReflect.Descriptor instead.\nfunc (*CoGroupMap) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{52}\n}\n\nfunc (x *CoGroupMap) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetInputGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.InputGroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetOther() *Relation {\n\tif x != nil {\n\t\treturn x.Other\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetOtherGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.OtherGroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetFunc() *CommonInlineUserDefinedFunction {\n\tif x != nil {\n\t\treturn x.Func\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetInputSortingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.InputSortingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *CoGroupMap) GetOtherSortingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.OtherSortingExpressions\n\t}\n\treturn nil\n}\n\ntype ApplyInPandasWithState struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation for applyInPandasWithState.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Expressions for grouping keys.\n\tGroupingExpressions []*Expression `protobuf:\"bytes,2,rep,name=grouping_expressions,json=groupingExpressions,proto3\" json:\"grouping_expressions,omitempty\"`\n\t// (Required) Input user-defined function.\n\tFunc *CommonInlineUserDefinedFunction `protobuf:\"bytes,3,opt,name=func,proto3\" json:\"func,omitempty\"`\n\t// (Required) Schema for the output DataFrame.\n\tOutputSchema string `protobuf:\"bytes,4,opt,name=output_schema,json=outputSchema,proto3\" json:\"output_schema,omitempty\"`\n\t// (Required) Schema for the state.\n\tStateSchema string `protobuf:\"bytes,5,opt,name=state_schema,json=stateSchema,proto3\" json:\"state_schema,omitempty\"`\n\t// (Required) The output mode of the function.\n\tOutputMode string `protobuf:\"bytes,6,opt,name=output_mode,json=outputMode,proto3\" json:\"output_mode,omitempty\"`\n\t// (Required) Timeout configuration for groups that do not receive data for a while.\n\tTimeoutConf string `protobuf:\"bytes,7,opt,name=timeout_conf,json=timeoutConf,proto3\" json:\"timeout_conf,omitempty\"`\n}\n\nfunc (x *ApplyInPandasWithState) Reset() {\n\t*x = ApplyInPandasWithState{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[53]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *ApplyInPandasWithState) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*ApplyInPandasWithState) ProtoMessage() {}\n\nfunc (x *ApplyInPandasWithState) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[53]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use ApplyInPandasWithState.ProtoReflect.Descriptor instead.\nfunc (*ApplyInPandasWithState) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{53}\n}\n\nfunc (x *ApplyInPandasWithState) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *ApplyInPandasWithState) GetGroupingExpressions() []*Expression {\n\tif x != nil {\n\t\treturn x.GroupingExpressions\n\t}\n\treturn nil\n}\n\nfunc (x *ApplyInPandasWithState) GetFunc() *CommonInlineUserDefinedFunction {\n\tif x != nil {\n\t\treturn x.Func\n\t}\n\treturn nil\n}\n\nfunc (x *ApplyInPandasWithState) GetOutputSchema() string {\n\tif x != nil {\n\t\treturn x.OutputSchema\n\t}\n\treturn \"\"\n}\n\nfunc (x *ApplyInPandasWithState) GetStateSchema() string {\n\tif x != nil {\n\t\treturn x.StateSchema\n\t}\n\treturn \"\"\n}\n\nfunc (x *ApplyInPandasWithState) GetOutputMode() string {\n\tif x != nil {\n\t\treturn x.OutputMode\n\t}\n\treturn \"\"\n}\n\nfunc (x *ApplyInPandasWithState) GetTimeoutConf() string {\n\tif x != nil {\n\t\treturn x.TimeoutConf\n\t}\n\treturn \"\"\n}\n\ntype CommonInlineUserDefinedTableFunction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Name of the user-defined table function.\n\tFunctionName string `protobuf:\"bytes,1,opt,name=function_name,json=functionName,proto3\" json:\"function_name,omitempty\"`\n\t// (Optional) Whether the user-defined table function is deterministic.\n\tDeterministic bool `protobuf:\"varint,2,opt,name=deterministic,proto3\" json:\"deterministic,omitempty\"`\n\t// (Optional) Function input arguments. Empty arguments are allowed.\n\tArguments []*Expression `protobuf:\"bytes,3,rep,name=arguments,proto3\" json:\"arguments,omitempty\"`\n\t// (Required) Type of the user-defined table function.\n\t//\n\t// Types that are assignable to Function:\n\t//\n\t//\t*CommonInlineUserDefinedTableFunction_PythonUdtf\n\tFunction isCommonInlineUserDefinedTableFunction_Function `protobuf_oneof:\"function\"`\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) Reset() {\n\t*x = CommonInlineUserDefinedTableFunction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[54]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CommonInlineUserDefinedTableFunction) ProtoMessage() {}\n\nfunc (x *CommonInlineUserDefinedTableFunction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[54]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CommonInlineUserDefinedTableFunction.ProtoReflect.Descriptor instead.\nfunc (*CommonInlineUserDefinedTableFunction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{54}\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) GetFunctionName() string {\n\tif x != nil {\n\t\treturn x.FunctionName\n\t}\n\treturn \"\"\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) GetDeterministic() bool {\n\tif x != nil {\n\t\treturn x.Deterministic\n\t}\n\treturn false\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) GetArguments() []*Expression {\n\tif x != nil {\n\t\treturn x.Arguments\n\t}\n\treturn nil\n}\n\nfunc (m *CommonInlineUserDefinedTableFunction) GetFunction() isCommonInlineUserDefinedTableFunction_Function {\n\tif m != nil {\n\t\treturn m.Function\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedTableFunction) GetPythonUdtf() *PythonUDTF {\n\tif x, ok := x.GetFunction().(*CommonInlineUserDefinedTableFunction_PythonUdtf); ok {\n\t\treturn x.PythonUdtf\n\t}\n\treturn nil\n}\n\ntype isCommonInlineUserDefinedTableFunction_Function interface {\n\tisCommonInlineUserDefinedTableFunction_Function()\n}\n\ntype CommonInlineUserDefinedTableFunction_PythonUdtf struct {\n\tPythonUdtf *PythonUDTF `protobuf:\"bytes,4,opt,name=python_udtf,json=pythonUdtf,proto3,oneof\"`\n}\n\nfunc (*CommonInlineUserDefinedTableFunction_PythonUdtf) isCommonInlineUserDefinedTableFunction_Function() {\n}\n\ntype PythonUDTF struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Return type of the Python UDTF.\n\tReturnType *DataType `protobuf:\"bytes,1,opt,name=return_type,json=returnType,proto3,oneof\" json:\"return_type,omitempty\"`\n\t// (Required) EvalType of the Python UDTF.\n\tEvalType int32 `protobuf:\"varint,2,opt,name=eval_type,json=evalType,proto3\" json:\"eval_type,omitempty\"`\n\t// (Required) The encoded commands of the Python UDTF.\n\tCommand []byte `protobuf:\"bytes,3,opt,name=command,proto3\" json:\"command,omitempty\"`\n\t// (Required) Python version being used in the client.\n\tPythonVer string `protobuf:\"bytes,4,opt,name=python_ver,json=pythonVer,proto3\" json:\"python_ver,omitempty\"`\n}\n\nfunc (x *PythonUDTF) Reset() {\n\t*x = PythonUDTF{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[55]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PythonUDTF) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PythonUDTF) ProtoMessage() {}\n\nfunc (x *PythonUDTF) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[55]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PythonUDTF.ProtoReflect.Descriptor instead.\nfunc (*PythonUDTF) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{55}\n}\n\nfunc (x *PythonUDTF) GetReturnType() *DataType {\n\tif x != nil {\n\t\treturn x.ReturnType\n\t}\n\treturn nil\n}\n\nfunc (x *PythonUDTF) GetEvalType() int32 {\n\tif x != nil {\n\t\treturn x.EvalType\n\t}\n\treturn 0\n}\n\nfunc (x *PythonUDTF) GetCommand() []byte {\n\tif x != nil {\n\t\treturn x.Command\n\t}\n\treturn nil\n}\n\nfunc (x *PythonUDTF) GetPythonVer() string {\n\tif x != nil {\n\t\treturn x.PythonVer\n\t}\n\treturn \"\"\n}\n\ntype CommonInlineUserDefinedDataSource struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Name of the data source.\n\tName string `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Required) The data source type.\n\t//\n\t// Types that are assignable to DataSource:\n\t//\n\t//\t*CommonInlineUserDefinedDataSource_PythonDataSource\n\tDataSource isCommonInlineUserDefinedDataSource_DataSource `protobuf_oneof:\"data_source\"`\n}\n\nfunc (x *CommonInlineUserDefinedDataSource) Reset() {\n\t*x = CommonInlineUserDefinedDataSource{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[56]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CommonInlineUserDefinedDataSource) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CommonInlineUserDefinedDataSource) ProtoMessage() {}\n\nfunc (x *CommonInlineUserDefinedDataSource) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[56]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CommonInlineUserDefinedDataSource.ProtoReflect.Descriptor instead.\nfunc (*CommonInlineUserDefinedDataSource) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{56}\n}\n\nfunc (x *CommonInlineUserDefinedDataSource) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (m *CommonInlineUserDefinedDataSource) GetDataSource() isCommonInlineUserDefinedDataSource_DataSource {\n\tif m != nil {\n\t\treturn m.DataSource\n\t}\n\treturn nil\n}\n\nfunc (x *CommonInlineUserDefinedDataSource) GetPythonDataSource() *PythonDataSource {\n\tif x, ok := x.GetDataSource().(*CommonInlineUserDefinedDataSource_PythonDataSource); ok {\n\t\treturn x.PythonDataSource\n\t}\n\treturn nil\n}\n\ntype isCommonInlineUserDefinedDataSource_DataSource interface {\n\tisCommonInlineUserDefinedDataSource_DataSource()\n}\n\ntype CommonInlineUserDefinedDataSource_PythonDataSource struct {\n\tPythonDataSource *PythonDataSource `protobuf:\"bytes,2,opt,name=python_data_source,json=pythonDataSource,proto3,oneof\"`\n}\n\nfunc (*CommonInlineUserDefinedDataSource_PythonDataSource) isCommonInlineUserDefinedDataSource_DataSource() {\n}\n\ntype PythonDataSource struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The encoded commands of the Python data source.\n\tCommand []byte `protobuf:\"bytes,1,opt,name=command,proto3\" json:\"command,omitempty\"`\n\t// (Required) Python version being used in the client.\n\tPythonVer string `protobuf:\"bytes,2,opt,name=python_ver,json=pythonVer,proto3\" json:\"python_ver,omitempty\"`\n}\n\nfunc (x *PythonDataSource) Reset() {\n\t*x = PythonDataSource{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[57]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *PythonDataSource) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*PythonDataSource) ProtoMessage() {}\n\nfunc (x *PythonDataSource) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[57]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use PythonDataSource.ProtoReflect.Descriptor instead.\nfunc (*PythonDataSource) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{57}\n}\n\nfunc (x *PythonDataSource) GetCommand() []byte {\n\tif x != nil {\n\t\treturn x.Command\n\t}\n\treturn nil\n}\n\nfunc (x *PythonDataSource) GetPythonVer() string {\n\tif x != nil {\n\t\treturn x.PythonVer\n\t}\n\treturn \"\"\n}\n\n// Collect arbitrary (named) metrics from a dataset.\ntype CollectMetrics struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The input relation.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) Name of the metrics.\n\tName string `protobuf:\"bytes,2,opt,name=name,proto3\" json:\"name,omitempty\"`\n\t// (Required) The metric sequence.\n\tMetrics []*Expression `protobuf:\"bytes,3,rep,name=metrics,proto3\" json:\"metrics,omitempty\"`\n}\n\nfunc (x *CollectMetrics) Reset() {\n\t*x = CollectMetrics{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[58]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *CollectMetrics) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*CollectMetrics) ProtoMessage() {}\n\nfunc (x *CollectMetrics) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[58]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use CollectMetrics.ProtoReflect.Descriptor instead.\nfunc (*CollectMetrics) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{58}\n}\n\nfunc (x *CollectMetrics) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *CollectMetrics) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *CollectMetrics) GetMetrics() []*Expression {\n\tif x != nil {\n\t\treturn x.Metrics\n\t}\n\treturn nil\n}\n\ntype Parse struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Input relation to Parse. The input is expected to have single text column.\n\tInput *Relation `protobuf:\"bytes,1,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// (Required) The expected format of the text.\n\tFormat Parse_ParseFormat `protobuf:\"varint,2,opt,name=format,proto3,enum=spark.connect.Parse_ParseFormat\" json:\"format,omitempty\"`\n\t// (Optional) DataType representing the schema. If not set, Spark will infer the schema.\n\tSchema *DataType `protobuf:\"bytes,3,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n\t// Options for the csv/json parser. The map key is case insensitive.\n\tOptions map[string]string `protobuf:\"bytes,4,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *Parse) Reset() {\n\t*x = Parse{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[59]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Parse) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Parse) ProtoMessage() {}\n\nfunc (x *Parse) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[59]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Parse.ProtoReflect.Descriptor instead.\nfunc (*Parse) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{59}\n}\n\nfunc (x *Parse) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *Parse) GetFormat() Parse_ParseFormat {\n\tif x != nil {\n\t\treturn x.Format\n\t}\n\treturn Parse_PARSE_FORMAT_UNSPECIFIED\n}\n\nfunc (x *Parse) GetSchema() *DataType {\n\tif x != nil {\n\t\treturn x.Schema\n\t}\n\treturn nil\n}\n\nfunc (x *Parse) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\n// Relation of type [[AsOfJoin]].\n//\n// `left` and `right` must be present.\ntype AsOfJoin struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Left input relation for a Join.\n\tLeft *Relation `protobuf:\"bytes,1,opt,name=left,proto3\" json:\"left,omitempty\"`\n\t// (Required) Right input relation for a Join.\n\tRight *Relation `protobuf:\"bytes,2,opt,name=right,proto3\" json:\"right,omitempty\"`\n\t// (Required) Field to join on in left DataFrame\n\tLeftAsOf *Expression `protobuf:\"bytes,3,opt,name=left_as_of,json=leftAsOf,proto3\" json:\"left_as_of,omitempty\"`\n\t// (Required) Field to join on in right DataFrame\n\tRightAsOf *Expression `protobuf:\"bytes,4,opt,name=right_as_of,json=rightAsOf,proto3\" json:\"right_as_of,omitempty\"`\n\t// (Optional) The join condition. Could be unset when `using_columns` is utilized.\n\t//\n\t// This field does not co-exist with using_columns.\n\tJoinExpr *Expression `protobuf:\"bytes,5,opt,name=join_expr,json=joinExpr,proto3\" json:\"join_expr,omitempty\"`\n\t// Optional. using_columns provides a list of columns that should present on both sides of\n\t// the join inputs that this Join will join on. For example A JOIN B USING col_name is\n\t// equivalent to A JOIN B on A.col_name = B.col_name.\n\t//\n\t// This field does not co-exist with join_condition.\n\tUsingColumns []string `protobuf:\"bytes,6,rep,name=using_columns,json=usingColumns,proto3\" json:\"using_columns,omitempty\"`\n\t// (Required) The join type.\n\tJoinType string `protobuf:\"bytes,7,opt,name=join_type,json=joinType,proto3\" json:\"join_type,omitempty\"`\n\t// (Optional) The asof tolerance within this range.\n\tTolerance *Expression `protobuf:\"bytes,8,opt,name=tolerance,proto3\" json:\"tolerance,omitempty\"`\n\t// (Required) Whether allow matching with the same value or not.\n\tAllowExactMatches bool `protobuf:\"varint,9,opt,name=allow_exact_matches,json=allowExactMatches,proto3\" json:\"allow_exact_matches,omitempty\"`\n\t// (Required) Whether to search for prior, subsequent, or closest matches.\n\tDirection string `protobuf:\"bytes,10,opt,name=direction,proto3\" json:\"direction,omitempty\"`\n}\n\nfunc (x *AsOfJoin) Reset() {\n\t*x = AsOfJoin{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[60]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *AsOfJoin) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*AsOfJoin) ProtoMessage() {}\n\nfunc (x *AsOfJoin) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[60]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use AsOfJoin.ProtoReflect.Descriptor instead.\nfunc (*AsOfJoin) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{60}\n}\n\nfunc (x *AsOfJoin) GetLeft() *Relation {\n\tif x != nil {\n\t\treturn x.Left\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetRight() *Relation {\n\tif x != nil {\n\t\treturn x.Right\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetLeftAsOf() *Expression {\n\tif x != nil {\n\t\treturn x.LeftAsOf\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetRightAsOf() *Expression {\n\tif x != nil {\n\t\treturn x.RightAsOf\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetJoinExpr() *Expression {\n\tif x != nil {\n\t\treturn x.JoinExpr\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetUsingColumns() []string {\n\tif x != nil {\n\t\treturn x.UsingColumns\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetJoinType() string {\n\tif x != nil {\n\t\treturn x.JoinType\n\t}\n\treturn \"\"\n}\n\nfunc (x *AsOfJoin) GetTolerance() *Expression {\n\tif x != nil {\n\t\treturn x.Tolerance\n\t}\n\treturn nil\n}\n\nfunc (x *AsOfJoin) GetAllowExactMatches() bool {\n\tif x != nil {\n\t\treturn x.AllowExactMatches\n\t}\n\treturn false\n}\n\nfunc (x *AsOfJoin) GetDirection() string {\n\tif x != nil {\n\t\treturn x.Direction\n\t}\n\treturn \"\"\n}\n\n// Relation of type [[LateralJoin]].\n//\n// `left` and `right` must be present.\ntype LateralJoin struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Left input relation for a Join.\n\tLeft *Relation `protobuf:\"bytes,1,opt,name=left,proto3\" json:\"left,omitempty\"`\n\t// (Required) Right input relation for a Join.\n\tRight *Relation `protobuf:\"bytes,2,opt,name=right,proto3\" json:\"right,omitempty\"`\n\t// (Optional) The join condition.\n\tJoinCondition *Expression `protobuf:\"bytes,3,opt,name=join_condition,json=joinCondition,proto3\" json:\"join_condition,omitempty\"`\n\t// (Required) The join type.\n\tJoinType Join_JoinType `protobuf:\"varint,4,opt,name=join_type,json=joinType,proto3,enum=spark.connect.Join_JoinType\" json:\"join_type,omitempty\"`\n}\n\nfunc (x *LateralJoin) Reset() {\n\t*x = LateralJoin{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[61]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *LateralJoin) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*LateralJoin) ProtoMessage() {}\n\nfunc (x *LateralJoin) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[61]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use LateralJoin.ProtoReflect.Descriptor instead.\nfunc (*LateralJoin) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{61}\n}\n\nfunc (x *LateralJoin) GetLeft() *Relation {\n\tif x != nil {\n\t\treturn x.Left\n\t}\n\treturn nil\n}\n\nfunc (x *LateralJoin) GetRight() *Relation {\n\tif x != nil {\n\t\treturn x.Right\n\t}\n\treturn nil\n}\n\nfunc (x *LateralJoin) GetJoinCondition() *Expression {\n\tif x != nil {\n\t\treturn x.JoinCondition\n\t}\n\treturn nil\n}\n\nfunc (x *LateralJoin) GetJoinType() Join_JoinType {\n\tif x != nil {\n\t\treturn x.JoinType\n\t}\n\treturn Join_JOIN_TYPE_UNSPECIFIED\n}\n\n// Relation to represent transform(input) of the operator\n// which could be a cached model or a new transformer\ntype MlRelation_Transform struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Operator:\n\t//\n\t//\t*MlRelation_Transform_ObjRef\n\t//\t*MlRelation_Transform_Transformer\n\tOperator isMlRelation_Transform_Operator `protobuf_oneof:\"operator\"`\n\t// the input dataframe\n\tInput *Relation `protobuf:\"bytes,3,opt,name=input,proto3\" json:\"input,omitempty\"`\n\t// the operator specific parameters\n\tParams *MlParams `protobuf:\"bytes,4,opt,name=params,proto3\" json:\"params,omitempty\"`\n}\n\nfunc (x *MlRelation_Transform) Reset() {\n\t*x = MlRelation_Transform{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[62]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *MlRelation_Transform) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*MlRelation_Transform) ProtoMessage() {}\n\nfunc (x *MlRelation_Transform) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[62]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use MlRelation_Transform.ProtoReflect.Descriptor instead.\nfunc (*MlRelation_Transform) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{1, 0}\n}\n\nfunc (m *MlRelation_Transform) GetOperator() isMlRelation_Transform_Operator {\n\tif m != nil {\n\t\treturn m.Operator\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation_Transform) GetObjRef() *ObjectRef {\n\tif x, ok := x.GetOperator().(*MlRelation_Transform_ObjRef); ok {\n\t\treturn x.ObjRef\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation_Transform) GetTransformer() *MlOperator {\n\tif x, ok := x.GetOperator().(*MlRelation_Transform_Transformer); ok {\n\t\treturn x.Transformer\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation_Transform) GetInput() *Relation {\n\tif x != nil {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\nfunc (x *MlRelation_Transform) GetParams() *MlParams {\n\tif x != nil {\n\t\treturn x.Params\n\t}\n\treturn nil\n}\n\ntype isMlRelation_Transform_Operator interface {\n\tisMlRelation_Transform_Operator()\n}\n\ntype MlRelation_Transform_ObjRef struct {\n\t// Object reference\n\tObjRef *ObjectRef `protobuf:\"bytes,1,opt,name=obj_ref,json=objRef,proto3,oneof\"`\n}\n\ntype MlRelation_Transform_Transformer struct {\n\t// Could be an ML transformer like VectorAssembler\n\tTransformer *MlOperator `protobuf:\"bytes,2,opt,name=transformer,proto3,oneof\"`\n}\n\nfunc (*MlRelation_Transform_ObjRef) isMlRelation_Transform_Operator() {}\n\nfunc (*MlRelation_Transform_Transformer) isMlRelation_Transform_Operator() {}\n\n// Represents a method with inclusion of method name and its arguments\ntype Fetch_Method struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) the method name\n\tMethod string `protobuf:\"bytes,1,opt,name=method,proto3\" json:\"method,omitempty\"`\n\t// (Optional) the arguments of the method\n\tArgs []*Fetch_Method_Args `protobuf:\"bytes,2,rep,name=args,proto3\" json:\"args,omitempty\"`\n}\n\nfunc (x *Fetch_Method) Reset() {\n\t*x = Fetch_Method{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[63]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Fetch_Method) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Fetch_Method) ProtoMessage() {}\n\nfunc (x *Fetch_Method) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[63]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Fetch_Method.ProtoReflect.Descriptor instead.\nfunc (*Fetch_Method) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{2, 0}\n}\n\nfunc (x *Fetch_Method) GetMethod() string {\n\tif x != nil {\n\t\treturn x.Method\n\t}\n\treturn \"\"\n}\n\nfunc (x *Fetch_Method) GetArgs() []*Fetch_Method_Args {\n\tif x != nil {\n\t\treturn x.Args\n\t}\n\treturn nil\n}\n\ntype Fetch_Method_Args struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to ArgsType:\n\t//\n\t//\t*Fetch_Method_Args_Param\n\t//\t*Fetch_Method_Args_Input\n\tArgsType isFetch_Method_Args_ArgsType `protobuf_oneof:\"args_type\"`\n}\n\nfunc (x *Fetch_Method_Args) Reset() {\n\t*x = Fetch_Method_Args{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[64]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Fetch_Method_Args) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Fetch_Method_Args) ProtoMessage() {}\n\nfunc (x *Fetch_Method_Args) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[64]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Fetch_Method_Args.ProtoReflect.Descriptor instead.\nfunc (*Fetch_Method_Args) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{2, 0, 0}\n}\n\nfunc (m *Fetch_Method_Args) GetArgsType() isFetch_Method_Args_ArgsType {\n\tif m != nil {\n\t\treturn m.ArgsType\n\t}\n\treturn nil\n}\n\nfunc (x *Fetch_Method_Args) GetParam() *Expression_Literal {\n\tif x, ok := x.GetArgsType().(*Fetch_Method_Args_Param); ok {\n\t\treturn x.Param\n\t}\n\treturn nil\n}\n\nfunc (x *Fetch_Method_Args) GetInput() *Relation {\n\tif x, ok := x.GetArgsType().(*Fetch_Method_Args_Input); ok {\n\t\treturn x.Input\n\t}\n\treturn nil\n}\n\ntype isFetch_Method_Args_ArgsType interface {\n\tisFetch_Method_Args_ArgsType()\n}\n\ntype Fetch_Method_Args_Param struct {\n\tParam *Expression_Literal `protobuf:\"bytes,1,opt,name=param,proto3,oneof\"`\n}\n\ntype Fetch_Method_Args_Input struct {\n\tInput *Relation `protobuf:\"bytes,2,opt,name=input,proto3,oneof\"`\n}\n\nfunc (*Fetch_Method_Args_Param) isFetch_Method_Args_ArgsType() {}\n\nfunc (*Fetch_Method_Args_Input) isFetch_Method_Args_ArgsType() {}\n\ntype Read_NamedTable struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Unparsed identifier for the table.\n\tUnparsedIdentifier string `protobuf:\"bytes,1,opt,name=unparsed_identifier,json=unparsedIdentifier,proto3\" json:\"unparsed_identifier,omitempty\"`\n\t// Options for the named table. The map key is case insensitive.\n\tOptions map[string]string `protobuf:\"bytes,2,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n}\n\nfunc (x *Read_NamedTable) Reset() {\n\t*x = Read_NamedTable{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[67]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Read_NamedTable) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Read_NamedTable) ProtoMessage() {}\n\nfunc (x *Read_NamedTable) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[67]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Read_NamedTable.ProtoReflect.Descriptor instead.\nfunc (*Read_NamedTable) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{7, 0}\n}\n\nfunc (x *Read_NamedTable) GetUnparsedIdentifier() string {\n\tif x != nil {\n\t\treturn x.UnparsedIdentifier\n\t}\n\treturn \"\"\n}\n\nfunc (x *Read_NamedTable) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\ntype Read_DataSource struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Optional) Supported formats include: parquet, orc, text, json, parquet, csv, avro.\n\t//\n\t// If not set, the value from SQL conf 'spark.sql.sources.default' will be used.\n\tFormat *string `protobuf:\"bytes,1,opt,name=format,proto3,oneof\" json:\"format,omitempty\"`\n\t// (Optional) If not set, Spark will infer the schema.\n\t//\n\t// This schema string should be either DDL-formatted or JSON-formatted.\n\tSchema *string `protobuf:\"bytes,2,opt,name=schema,proto3,oneof\" json:\"schema,omitempty\"`\n\t// Options for the data source. The context of this map varies based on the\n\t// data source format. This options could be empty for valid data source format.\n\t// The map key is case insensitive.\n\tOptions map[string]string `protobuf:\"bytes,3,rep,name=options,proto3\" json:\"options,omitempty\" protobuf_key:\"bytes,1,opt,name=key,proto3\" protobuf_val:\"bytes,2,opt,name=value,proto3\"`\n\t// (Optional) A list of path for file-system backed data sources.\n\tPaths []string `protobuf:\"bytes,4,rep,name=paths,proto3\" json:\"paths,omitempty\"`\n\t// (Optional) Condition in the where clause for each partition.\n\t//\n\t// This is only supported by the JDBC data source.\n\tPredicates []string `protobuf:\"bytes,5,rep,name=predicates,proto3\" json:\"predicates,omitempty\"`\n}\n\nfunc (x *Read_DataSource) Reset() {\n\t*x = Read_DataSource{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[68]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Read_DataSource) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Read_DataSource) ProtoMessage() {}\n\nfunc (x *Read_DataSource) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[68]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Read_DataSource.ProtoReflect.Descriptor instead.\nfunc (*Read_DataSource) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{7, 1}\n}\n\nfunc (x *Read_DataSource) GetFormat() string {\n\tif x != nil && x.Format != nil {\n\t\treturn *x.Format\n\t}\n\treturn \"\"\n}\n\nfunc (x *Read_DataSource) GetSchema() string {\n\tif x != nil && x.Schema != nil {\n\t\treturn *x.Schema\n\t}\n\treturn \"\"\n}\n\nfunc (x *Read_DataSource) GetOptions() map[string]string {\n\tif x != nil {\n\t\treturn x.Options\n\t}\n\treturn nil\n}\n\nfunc (x *Read_DataSource) GetPaths() []string {\n\tif x != nil {\n\t\treturn x.Paths\n\t}\n\treturn nil\n}\n\nfunc (x *Read_DataSource) GetPredicates() []string {\n\tif x != nil {\n\t\treturn x.Predicates\n\t}\n\treturn nil\n}\n\ntype Join_JoinDataType struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// If the left data type is a struct.\n\tIsLeftStruct bool `protobuf:\"varint,1,opt,name=is_left_struct,json=isLeftStruct,proto3\" json:\"is_left_struct,omitempty\"`\n\t// If the right data type is a struct.\n\tIsRightStruct bool `protobuf:\"varint,2,opt,name=is_right_struct,json=isRightStruct,proto3\" json:\"is_right_struct,omitempty\"`\n}\n\nfunc (x *Join_JoinDataType) Reset() {\n\t*x = Join_JoinDataType{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[71]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Join_JoinDataType) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Join_JoinDataType) ProtoMessage() {}\n\nfunc (x *Join_JoinDataType) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[71]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Join_JoinDataType.ProtoReflect.Descriptor instead.\nfunc (*Join_JoinDataType) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{10, 0}\n}\n\nfunc (x *Join_JoinDataType) GetIsLeftStruct() bool {\n\tif x != nil {\n\t\treturn x.IsLeftStruct\n\t}\n\treturn false\n}\n\nfunc (x *Join_JoinDataType) GetIsRightStruct() bool {\n\tif x != nil {\n\t\treturn x.IsRightStruct\n\t}\n\treturn false\n}\n\ntype Aggregate_Pivot struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The column to pivot\n\tCol *Expression `protobuf:\"bytes,1,opt,name=col,proto3\" json:\"col,omitempty\"`\n\t// (Optional) List of values that will be translated to columns in the output DataFrame.\n\t//\n\t// Note that if it is empty, the server side will immediately trigger a job to collect\n\t// the distinct values of the column.\n\tValues []*Expression_Literal `protobuf:\"bytes,2,rep,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Aggregate_Pivot) Reset() {\n\t*x = Aggregate_Pivot{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[72]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Aggregate_Pivot) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Aggregate_Pivot) ProtoMessage() {}\n\nfunc (x *Aggregate_Pivot) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[72]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Aggregate_Pivot.ProtoReflect.Descriptor instead.\nfunc (*Aggregate_Pivot) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{15, 0}\n}\n\nfunc (x *Aggregate_Pivot) GetCol() *Expression {\n\tif x != nil {\n\t\treturn x.Col\n\t}\n\treturn nil\n}\n\nfunc (x *Aggregate_Pivot) GetValues() []*Expression_Literal {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\ntype Aggregate_GroupingSets struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) Individual grouping set\n\tGroupingSet []*Expression `protobuf:\"bytes,1,rep,name=grouping_set,json=groupingSet,proto3\" json:\"grouping_set,omitempty\"`\n}\n\nfunc (x *Aggregate_GroupingSets) Reset() {\n\t*x = Aggregate_GroupingSets{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[73]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Aggregate_GroupingSets) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Aggregate_GroupingSets) ProtoMessage() {}\n\nfunc (x *Aggregate_GroupingSets) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[73]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Aggregate_GroupingSets.ProtoReflect.Descriptor instead.\nfunc (*Aggregate_GroupingSets) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{15, 1}\n}\n\nfunc (x *Aggregate_GroupingSets) GetGroupingSet() []*Expression {\n\tif x != nil {\n\t\treturn x.GroupingSet\n\t}\n\treturn nil\n}\n\ntype StatSampleBy_Fraction struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The stratum.\n\tStratum *Expression_Literal `protobuf:\"bytes,1,opt,name=stratum,proto3\" json:\"stratum,omitempty\"`\n\t// (Required) The fraction value. Must be in [0, 1].\n\tFraction float64 `protobuf:\"fixed64,2,opt,name=fraction,proto3\" json:\"fraction,omitempty\"`\n}\n\nfunc (x *StatSampleBy_Fraction) Reset() {\n\t*x = StatSampleBy_Fraction{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[74]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *StatSampleBy_Fraction) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*StatSampleBy_Fraction) ProtoMessage() {}\n\nfunc (x *StatSampleBy_Fraction) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[74]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use StatSampleBy_Fraction.ProtoReflect.Descriptor instead.\nfunc (*StatSampleBy_Fraction) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{35, 0}\n}\n\nfunc (x *StatSampleBy_Fraction) GetStratum() *Expression_Literal {\n\tif x != nil {\n\t\treturn x.Stratum\n\t}\n\treturn nil\n}\n\nfunc (x *StatSampleBy_Fraction) GetFraction() float64 {\n\tif x != nil {\n\t\treturn x.Fraction\n\t}\n\treturn 0\n}\n\ntype NAReplace_Replacement struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The old value.\n\t//\n\t// Only 4 data types are supported now: null, bool, double, string.\n\tOldValue *Expression_Literal `protobuf:\"bytes,1,opt,name=old_value,json=oldValue,proto3\" json:\"old_value,omitempty\"`\n\t// (Required) The new value.\n\t//\n\t// Should be of the same data type with the old value.\n\tNewValue *Expression_Literal `protobuf:\"bytes,2,opt,name=new_value,json=newValue,proto3\" json:\"new_value,omitempty\"`\n}\n\nfunc (x *NAReplace_Replacement) Reset() {\n\t*x = NAReplace_Replacement{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[75]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *NAReplace_Replacement) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*NAReplace_Replacement) ProtoMessage() {}\n\nfunc (x *NAReplace_Replacement) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[75]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use NAReplace_Replacement.ProtoReflect.Descriptor instead.\nfunc (*NAReplace_Replacement) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{38, 0}\n}\n\nfunc (x *NAReplace_Replacement) GetOldValue() *Expression_Literal {\n\tif x != nil {\n\t\treturn x.OldValue\n\t}\n\treturn nil\n}\n\nfunc (x *NAReplace_Replacement) GetNewValue() *Expression_Literal {\n\tif x != nil {\n\t\treturn x.NewValue\n\t}\n\treturn nil\n}\n\ntype WithColumnsRenamed_Rename struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The existing column name.\n\tColName string `protobuf:\"bytes,1,opt,name=col_name,json=colName,proto3\" json:\"col_name,omitempty\"`\n\t// (Required) The new column name.\n\tNewColName string `protobuf:\"bytes,2,opt,name=new_col_name,json=newColName,proto3\" json:\"new_col_name,omitempty\"`\n}\n\nfunc (x *WithColumnsRenamed_Rename) Reset() {\n\t*x = WithColumnsRenamed_Rename{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[77]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *WithColumnsRenamed_Rename) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*WithColumnsRenamed_Rename) ProtoMessage() {}\n\nfunc (x *WithColumnsRenamed_Rename) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[77]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use WithColumnsRenamed_Rename.ProtoReflect.Descriptor instead.\nfunc (*WithColumnsRenamed_Rename) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{40, 1}\n}\n\nfunc (x *WithColumnsRenamed_Rename) GetColName() string {\n\tif x != nil {\n\t\treturn x.ColName\n\t}\n\treturn \"\"\n}\n\nfunc (x *WithColumnsRenamed_Rename) GetNewColName() string {\n\tif x != nil {\n\t\treturn x.NewColName\n\t}\n\treturn \"\"\n}\n\ntype Unpivot_Values struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tValues []*Expression `protobuf:\"bytes,1,rep,name=values,proto3\" json:\"values,omitempty\"`\n}\n\nfunc (x *Unpivot_Values) Reset() {\n\t*x = Unpivot_Values{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_relations_proto_msgTypes[78]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *Unpivot_Values) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*Unpivot_Values) ProtoMessage() {}\n\nfunc (x *Unpivot_Values) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_relations_proto_msgTypes[78]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use Unpivot_Values.ProtoReflect.Descriptor instead.\nfunc (*Unpivot_Values) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_relations_proto_rawDescGZIP(), []int{44, 0}\n}\n\nfunc (x *Unpivot_Values) GetValues() []*Expression {\n\tif x != nil {\n\t\treturn x.Values\n\t}\n\treturn nil\n}\n\nvar File_spark_connect_relations_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_relations_proto_rawDesc = []byte{\n\t0x0a, 0x1d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,\n\t0x0d, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x1a, 0x19,\n\t0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f,\n\t0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2f, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x2e, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x1a, 0x1a, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x6d, 0x6c,\n\t0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9c, 0x1d,\n\t0x0a, 0x08, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x06, 0x63, 0x6f,\n\t0x6d, 0x6d, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x52, 0x06, 0x63, 0x6f, 0x6d, 0x6d, 0x6f,\n\t0x6e, 0x12, 0x29, 0x0a, 0x04, 0x72, 0x65, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x61, 0x64, 0x48, 0x00, 0x52, 0x04, 0x72, 0x65, 0x61, 0x64, 0x12, 0x32, 0x0a, 0x07,\n\t0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x72,\n\t0x6f, 0x6a, 0x65, 0x63, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74,\n\t0x12, 0x2f, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x48, 0x00, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65,\n\t0x72, 0x12, 0x29, 0x0a, 0x04, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x4a, 0x6f, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x04, 0x6a, 0x6f, 0x69, 0x6e, 0x12, 0x34, 0x0a, 0x06,\n\t0x73, 0x65, 0x74, 0x5f, 0x6f, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x65, 0x74,\n\t0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x05, 0x73, 0x65, 0x74,\n\t0x4f, 0x70, 0x12, 0x29, 0x0a, 0x04, 0x73, 0x6f, 0x72, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x6f, 0x72, 0x74, 0x48, 0x00, 0x52, 0x04, 0x73, 0x6f, 0x72, 0x74, 0x12, 0x2c, 0x0a,\n\t0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x69, 0x6d,\n\t0x69, 0x74, 0x48, 0x00, 0x52, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x38, 0x0a, 0x09, 0x61,\n\t0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41,\n\t0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x09, 0x61, 0x67, 0x67, 0x72,\n\t0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x26, 0x0a, 0x03, 0x73, 0x71, 0x6c, 0x18, 0x0a, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x12, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x53, 0x51, 0x4c, 0x48, 0x00, 0x52, 0x03, 0x73, 0x71, 0x6c, 0x12, 0x45, 0x0a,\n\t0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18,\n\t0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0d, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x06, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x18, 0x0c,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x06, 0x73,\n\t0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x2f, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18,\n\t0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x48, 0x00, 0x52, 0x06,\n\t0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x3e, 0x0a, 0x0b, 0x64, 0x65, 0x64, 0x75, 0x70, 0x6c,\n\t0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x65, 0x64, 0x75,\n\t0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x65, 0x64, 0x75, 0x70,\n\t0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x2c, 0x0a, 0x05, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18,\n\t0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x05, 0x72,\n\t0x61, 0x6e, 0x67, 0x65, 0x12, 0x45, 0x0a, 0x0e, 0x73, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79,\n\t0x5f, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x75, 0x62,\n\t0x71, 0x75, 0x65, 0x72, 0x79, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x75,\n\t0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x3e, 0x0a, 0x0b, 0x72,\n\t0x65, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x52, 0x65, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b,\n\t0x72, 0x65, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x0a, 0x05, 0x74,\n\t0x6f, 0x5f, 0x64, 0x66, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x6f, 0x44, 0x46, 0x48,\n\t0x00, 0x52, 0x04, 0x74, 0x6f, 0x44, 0x66, 0x12, 0x55, 0x0a, 0x14, 0x77, 0x69, 0x74, 0x68, 0x5f,\n\t0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x18,\n\t0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x73, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x48, 0x00, 0x52, 0x12, 0x77, 0x69, 0x74, 0x68,\n\t0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x12, 0x3c,\n\t0x0a, 0x0b, 0x73, 0x68, 0x6f, 0x77, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x14, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x48, 0x00,\n\t0x52, 0x0a, 0x73, 0x68, 0x6f, 0x77, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x29, 0x0a, 0x04,\n\t0x64, 0x72, 0x6f, 0x70, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x48,\n\t0x00, 0x52, 0x04, 0x64, 0x72, 0x6f, 0x70, 0x12, 0x29, 0x0a, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x18,\n\t0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x61, 0x69, 0x6c, 0x48, 0x00, 0x52, 0x04, 0x74, 0x61,\n\t0x69, 0x6c, 0x12, 0x3f, 0x0a, 0x0c, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c,\n\t0x75, 0x6d, 0x6e, 0x73, 0x48, 0x00, 0x52, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x73, 0x12, 0x29, 0x0a, 0x04, 0x68, 0x69, 0x6e, 0x74, 0x18, 0x18, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x13, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x48, 0x69, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x04, 0x68, 0x69, 0x6e, 0x74, 0x12, 0x32,\n\t0x0a, 0x07, 0x75, 0x6e, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x16, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x55, 0x6e, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x48, 0x00, 0x52, 0x07, 0x75, 0x6e, 0x70, 0x69, 0x76,\n\t0x6f, 0x74, 0x12, 0x36, 0x0a, 0x09, 0x74, 0x6f, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18,\n\t0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x6f, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x48, 0x00,\n\t0x52, 0x08, 0x74, 0x6f, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x64, 0x0a, 0x19, 0x72, 0x65,\n\t0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x62, 0x79, 0x5f, 0x65, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x17, 0x72, 0x65, 0x70, 0x61, 0x72, 0x74, 0x69,\n\t0x74, 0x69, 0x6f, 0x6e, 0x42, 0x79, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x12, 0x45, 0x0a, 0x0e, 0x6d, 0x61, 0x70, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,\n\t0x6e, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x61, 0x70, 0x50, 0x61, 0x72, 0x74,\n\t0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x48, 0x00, 0x52, 0x0d, 0x6d, 0x61, 0x70, 0x50, 0x61, 0x72,\n\t0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x48, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65,\n\t0x63, 0x74, 0x5f, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x48,\n\t0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63,\n\t0x73, 0x12, 0x2c, 0x0a, 0x05, 0x70, 0x61, 0x72, 0x73, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x14, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x50, 0x61, 0x72, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x70, 0x61, 0x72, 0x73, 0x65, 0x12,\n\t0x36, 0x0a, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x1f, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4d, 0x61, 0x70, 0x48, 0x00, 0x52, 0x08, 0x67,\n\t0x72, 0x6f, 0x75, 0x70, 0x4d, 0x61, 0x70, 0x12, 0x3d, 0x0a, 0x0c, 0x63, 0x6f, 0x5f, 0x67, 0x72,\n\t0x6f, 0x75, 0x70, 0x5f, 0x6d, 0x61, 0x70, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f,\n\t0x47, 0x72, 0x6f, 0x75, 0x70, 0x4d, 0x61, 0x70, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x47, 0x72,\n\t0x6f, 0x75, 0x70, 0x4d, 0x61, 0x70, 0x12, 0x45, 0x0a, 0x0e, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x77,\n\t0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57,\n\t0x69, 0x74, 0x68, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x48, 0x00, 0x52, 0x0d,\n\t0x77, 0x69, 0x74, 0x68, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x12, 0x63, 0x0a,\n\t0x1a, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x5f, 0x69, 0x6e, 0x5f, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x73,\n\t0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x22, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x25, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x49, 0x6e, 0x50, 0x61, 0x6e, 0x64, 0x61, 0x73, 0x57,\n\t0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x16, 0x61, 0x70, 0x70, 0x6c,\n\t0x79, 0x49, 0x6e, 0x50, 0x61, 0x6e, 0x64, 0x61, 0x73, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61,\n\t0x74, 0x65, 0x12, 0x3c, 0x0a, 0x0b, 0x68, 0x74, 0x6d, 0x6c, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e,\n\t0x67, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x48, 0x74, 0x6d, 0x6c, 0x53, 0x74, 0x72, 0x69,\n\t0x6e, 0x67, 0x48, 0x00, 0x52, 0x0a, 0x68, 0x74, 0x6d, 0x6c, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67,\n\t0x12, 0x58, 0x0a, 0x15, 0x63, 0x61, 0x63, 0x68, 0x65, 0x64, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x6c,\n\t0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x43, 0x61, 0x63, 0x68, 0x65, 0x64, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x13, 0x63, 0x61, 0x63, 0x68, 0x65, 0x64, 0x4c, 0x6f, 0x63,\n\t0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5b, 0x0a, 0x16, 0x63, 0x61,\n\t0x63, 0x68, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65,\n\t0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48,\n\t0x00, 0x52, 0x14, 0x63, 0x61, 0x63, 0x68, 0x65, 0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x8e, 0x01, 0x0a, 0x29, 0x63, 0x6f, 0x6d, 0x6d,\n\t0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x64,\n\t0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x66, 0x75, 0x6e,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d,\n\t0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69,\n\t0x6e, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x48, 0x00, 0x52, 0x24, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65,\n\t0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65,\n\t0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x0a, 0x61, 0x73, 0x5f, 0x6f,\n\t0x66, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x73, 0x4f,\n\t0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x08, 0x61, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69,\n\t0x6e, 0x12, 0x85, 0x01, 0x0a, 0x26, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x6c,\n\t0x69, 0x6e, 0x65, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64,\n\t0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x28, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x30, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55,\n\t0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x48, 0x00, 0x52, 0x21, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e,\n\t0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x44,\n\t0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x45, 0x0a, 0x0e, 0x77, 0x69, 0x74,\n\t0x68, 0x5f, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x29, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x48,\n\t0x00, 0x52, 0x0d, 0x77, 0x69, 0x74, 0x68, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x12, 0x38, 0x0a, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x73, 0x65, 0x18, 0x2a, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x73, 0x65, 0x48, 0x00, 0x52,\n\t0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x6f, 0x73, 0x65, 0x12, 0x77, 0x0a, 0x20, 0x75, 0x6e,\n\t0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x64, 0x5f, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x2b,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64, 0x54,\n\t0x61, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x48, 0x00, 0x52, 0x1d, 0x75, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x64,\n\t0x54, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x3f, 0x0a, 0x0c, 0x6c, 0x61, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x5f, 0x6a,\n\t0x6f, 0x69, 0x6e, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4c, 0x61, 0x74, 0x65, 0x72, 0x61,\n\t0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x0b, 0x6c, 0x61, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x30, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x6c, 0x5f, 0x6e, 0x61, 0x18,\n\t0x5a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4e, 0x41, 0x46, 0x69, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x06,\n\t0x66, 0x69, 0x6c, 0x6c, 0x4e, 0x61, 0x12, 0x30, 0x0a, 0x07, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x6e,\n\t0x61, 0x18, 0x5b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4e, 0x41, 0x44, 0x72, 0x6f, 0x70, 0x48, 0x00,\n\t0x52, 0x06, 0x64, 0x72, 0x6f, 0x70, 0x4e, 0x61, 0x12, 0x34, 0x0a, 0x07, 0x72, 0x65, 0x70, 0x6c,\n\t0x61, 0x63, 0x65, 0x18, 0x5c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4e, 0x41, 0x52, 0x65, 0x70, 0x6c,\n\t0x61, 0x63, 0x65, 0x48, 0x00, 0x52, 0x07, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x12, 0x36,\n\t0x0a, 0x07, 0x73, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x18, 0x64, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1a, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x53, 0x74, 0x61, 0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x48, 0x00, 0x52, 0x07, 0x73,\n\t0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x39, 0x0a, 0x08, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x74,\n\t0x61, 0x62, 0x18, 0x65, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x43, 0x72, 0x6f,\n\t0x73, 0x73, 0x74, 0x61, 0x62, 0x48, 0x00, 0x52, 0x08, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x74, 0x61,\n\t0x62, 0x12, 0x39, 0x0a, 0x08, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x18, 0x66, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,\n\t0x48, 0x00, 0x52, 0x08, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x2a, 0x0a, 0x03,\n\t0x63, 0x6f, 0x76, 0x18, 0x67, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x43, 0x6f,\n\t0x76, 0x48, 0x00, 0x52, 0x03, 0x63, 0x6f, 0x76, 0x12, 0x2d, 0x0a, 0x04, 0x63, 0x6f, 0x72, 0x72,\n\t0x18, 0x68, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x43, 0x6f, 0x72, 0x72, 0x48,\n\t0x00, 0x52, 0x04, 0x63, 0x6f, 0x72, 0x72, 0x12, 0x4c, 0x0a, 0x0f, 0x61, 0x70, 0x70, 0x72, 0x6f,\n\t0x78, 0x5f, 0x71, 0x75, 0x61, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x69, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x74, 0x61, 0x74, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x51, 0x75, 0x61, 0x6e, 0x74,\n\t0x69, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x51, 0x75, 0x61,\n\t0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x3d, 0x0a, 0x0a, 0x66, 0x72, 0x65, 0x71, 0x5f, 0x69, 0x74,\n\t0x65, 0x6d, 0x73, 0x18, 0x6a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x46, 0x72,\n\t0x65, 0x71, 0x49, 0x74, 0x65, 0x6d, 0x73, 0x48, 0x00, 0x52, 0x09, 0x66, 0x72, 0x65, 0x71, 0x49,\n\t0x74, 0x65, 0x6d, 0x73, 0x12, 0x3a, 0x0a, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x62,\n\t0x79, 0x18, 0x6b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x53, 0x61, 0x6d, 0x70,\n\t0x6c, 0x65, 0x42, 0x79, 0x48, 0x00, 0x52, 0x08, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x42, 0x79,\n\t0x12, 0x33, 0x0a, 0x07, 0x63, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x18, 0xc8, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x16, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x48, 0x00, 0x52, 0x07, 0x63, 0x61,\n\t0x74, 0x61, 0x6c, 0x6f, 0x67, 0x12, 0x3d, 0x0a, 0x0b, 0x6d, 0x6c, 0x5f, 0x72, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x18, 0xac, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0a, 0x6d, 0x6c, 0x52, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f,\n\t0x6e, 0x18, 0xe6, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,\n\t0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x48, 0x00,\n\t0x52, 0x09, 0x65, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x33, 0x0a, 0x07, 0x75,\n\t0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x18, 0xe7, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x6e,\n\t0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e,\n\t0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xf8, 0x02, 0x0a,\n\t0x0a, 0x4d, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x09, 0x74,\n\t0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d,\n\t0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66,\n\t0x6f, 0x72, 0x6d, 0x48, 0x00, 0x52, 0x09, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d,\n\t0x12, 0x2c, 0x0a, 0x05, 0x66, 0x65, 0x74, 0x63, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x14, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x46, 0x65, 0x74, 0x63, 0x68, 0x48, 0x00, 0x52, 0x05, 0x66, 0x65, 0x74, 0x63, 0x68, 0x1a, 0xeb,\n\t0x01, 0x0a, 0x09, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x33, 0x0a, 0x07,\n\t0x6f, 0x62, 0x6a, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x62,\n\t0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x66, 0x48, 0x00, 0x52, 0x06, 0x6f, 0x62, 0x6a, 0x52, 0x65,\n\t0x66, 0x12, 0x3d, 0x0a, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4d, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f,\n\t0x72, 0x48, 0x00, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x2f, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x4d, 0x6c, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73,\n\t0x42, 0x0a, 0x0a, 0x08, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x6f, 0x72, 0x42, 0x09, 0x0a, 0x07,\n\t0x6d, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0xcb, 0x02, 0x0a, 0x05, 0x46, 0x65, 0x74, 0x63,\n\t0x68, 0x12, 0x31, 0x0a, 0x07, 0x6f, 0x62, 0x6a, 0x5f, 0x72, 0x65, 0x66, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x18, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x52, 0x65, 0x66, 0x52, 0x06, 0x6f, 0x62,\n\t0x6a, 0x52, 0x65, 0x66, 0x12, 0x35, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18,\n\t0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x2e, 0x4d, 0x65, 0x74, 0x68,\n\t0x6f, 0x64, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x1a, 0xd7, 0x01, 0x0a, 0x06,\n\t0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x34,\n\t0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x46, 0x65, 0x74,\n\t0x63, 0x68, 0x2e, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x2e, 0x41, 0x72, 0x67, 0x73, 0x52, 0x04,\n\t0x61, 0x72, 0x67, 0x73, 0x1a, 0x7f, 0x0a, 0x04, 0x41, 0x72, 0x67, 0x73, 0x12, 0x39, 0x0a, 0x05,\n\t0x70, 0x61, 0x72, 0x61, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x48, 0x00,\n\t0x52, 0x05, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x12, 0x2f, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48,\n\t0x00, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x42, 0x0b, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x73,\n\t0x5f, 0x74, 0x79, 0x70, 0x65, 0x22, 0x09, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e,\n\t0x22, 0x8e, 0x01, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x6e,\n\t0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x52, 0x0a, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x07, 0x70, 0x6c, 0x61, 0x6e,\n\t0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x06, 0x70, 0x6c, 0x61,\n\t0x6e, 0x49, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2d, 0x0a, 0x06, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x52, 0x06, 0x6f,\n\t0x72, 0x69, 0x67, 0x69, 0x6e, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x70, 0x6c, 0x61, 0x6e, 0x5f, 0x69,\n\t0x64, 0x22, 0xde, 0x03, 0x0a, 0x03, 0x53, 0x51, 0x4c, 0x12, 0x14, 0x0a, 0x05, 0x71, 0x75, 0x65,\n\t0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x71, 0x75, 0x65, 0x72, 0x79, 0x12,\n\t0x34, 0x0a, 0x04, 0x61, 0x72, 0x67, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x51,\n\t0x4c, 0x2e, 0x41, 0x72, 0x67, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x42, 0x02, 0x18, 0x01, 0x52,\n\t0x04, 0x61, 0x72, 0x67, 0x73, 0x12, 0x40, 0x0a, 0x08, 0x70, 0x6f, 0x73, 0x5f, 0x61, 0x72, 0x67,\n\t0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x42, 0x02, 0x18, 0x01, 0x52, 0x07,\n\t0x70, 0x6f, 0x73, 0x41, 0x72, 0x67, 0x73, 0x12, 0x4f, 0x0a, 0x0f, 0x6e, 0x61, 0x6d, 0x65, 0x64,\n\t0x5f, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x26, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x53, 0x51, 0x4c, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67, 0x75, 0x6d, 0x65,\n\t0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x41,\n\t0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x3e, 0x0a, 0x0d, 0x70, 0x6f, 0x73, 0x5f,\n\t0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x70, 0x6f, 0x73, 0x41,\n\t0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x5a, 0x0a, 0x09, 0x41, 0x72, 0x67, 0x73,\n\t0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,\n\t0x3a, 0x02, 0x38, 0x01, 0x1a, 0x5c, 0x0a, 0x13, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x41, 0x72, 0x67,\n\t0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,\n\t0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2f, 0x0a,\n\t0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02,\n\t0x38, 0x01, 0x22, 0x75, 0x0a, 0x0d, 0x57, 0x69, 0x74, 0x68, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x73, 0x12, 0x2b, 0x0a, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x72, 0x6f, 0x6f, 0x74,\n\t0x12, 0x37, 0x0a, 0x0a, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x18, 0x02,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x72,\n\t0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x22, 0x97, 0x05, 0x0a, 0x04, 0x52, 0x65,\n\t0x61, 0x64, 0x12, 0x41, 0x0a, 0x0b, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x5f, 0x74, 0x61, 0x62, 0x6c,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x2e, 0x4e, 0x61, 0x6d,\n\t0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x6e, 0x61, 0x6d, 0x65, 0x64,\n\t0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x41, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f,\n\t0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x2e,\n\t0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x64, 0x61,\n\t0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73,\n\t0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b,\n\t0x69, 0x73, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x1a, 0xc0, 0x01, 0x0a, 0x0a,\n\t0x4e, 0x61, 0x6d, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x2f, 0x0a, 0x13, 0x75, 0x6e,\n\t0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65,\n\t0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x75, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65,\n\t0x64, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x45, 0x0a, 0x07, 0x6f,\n\t0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61,\n\t0x64, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x2e, 0x4f, 0x70, 0x74,\n\t0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74,\n\t0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x95,\n\t0x02, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x1b, 0x0a,\n\t0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,\n\t0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x73, 0x63,\n\t0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x73, 0x63,\n\t0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x45, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f,\n\t0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x2e, 0x44, 0x61,\n\t0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x14,\n\t0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70,\n\t0x61, 0x74, 0x68, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63, 0x61, 0x74,\n\t0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x64, 0x69, 0x63,\n\t0x61, 0x74, 0x65, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45,\n\t0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,\n\t0x42, 0x09, 0x0a, 0x07, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x42, 0x09, 0x0a, 0x07, 0x5f,\n\t0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42, 0x0b, 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x22, 0x75, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x12, 0x2d,\n\t0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x3b, 0x0a,\n\t0x0b, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03,\n\t0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x65,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x70, 0x0a, 0x06, 0x46, 0x69,\n\t0x6c, 0x74, 0x65, 0x72, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x12, 0x37, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x95, 0x05, 0x0a,\n\t0x04, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x6c, 0x65,\n\t0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x72, 0x69, 0x67, 0x68,\n\t0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x63, 0x6f, 0x6e, 0x64, 0x69, 0x74,\n\t0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6a, 0x6f, 0x69, 0x6e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x09, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65,\n\t0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4a, 0x6f, 0x69, 0x6e, 0x2e, 0x4a, 0x6f, 0x69, 0x6e,\n\t0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6a, 0x6f, 0x69, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x23,\n\t0x0a, 0x0d, 0x75, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18,\n\t0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x75, 0x73, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x73, 0x12, 0x4b, 0x0a, 0x0e, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61,\n\t0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4a, 0x6f, 0x69, 0x6e,\n\t0x2e, 0x4a, 0x6f, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52,\n\t0x0c, 0x6a, 0x6f, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01,\n\t0x1a, 0x5c, 0x0a, 0x0c, 0x4a, 0x6f, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65,\n\t0x12, 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x75,\n\t0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x4c, 0x65, 0x66, 0x74,\n\t0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x73, 0x5f, 0x72, 0x69, 0x67,\n\t0x68, 0x74, 0x5f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52,\n\t0x0d, 0x69, 0x73, 0x52, 0x69, 0x67, 0x68, 0x74, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x22, 0xd0,\n\t0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x4a,\n\t0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49,\n\t0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x4f, 0x49, 0x4e, 0x5f, 0x54,\n\t0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x4a,\n\t0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x55, 0x4c, 0x4c, 0x5f, 0x4f, 0x55,\n\t0x54, 0x45, 0x52, 0x10, 0x02, 0x12, 0x18, 0x0a, 0x14, 0x4a, 0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59,\n\t0x50, 0x45, 0x5f, 0x4c, 0x45, 0x46, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x10, 0x03, 0x12,\n\t0x19, 0x0a, 0x15, 0x4a, 0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x52, 0x49, 0x47,\n\t0x48, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x45, 0x52, 0x10, 0x04, 0x12, 0x17, 0x0a, 0x13, 0x4a, 0x4f,\n\t0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4c, 0x45, 0x46, 0x54, 0x5f, 0x41, 0x4e, 0x54,\n\t0x49, 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, 0x4a, 0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45,\n\t0x5f, 0x4c, 0x45, 0x46, 0x54, 0x5f, 0x53, 0x45, 0x4d, 0x49, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f,\n\t0x4a, 0x4f, 0x49, 0x4e, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x52, 0x4f, 0x53, 0x53, 0x10,\n\t0x07, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f,\n\t0x74, 0x79, 0x70, 0x65, 0x22, 0xdf, 0x03, 0x0a, 0x0c, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x65, 0x72,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x0a, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x09, 0x6c, 0x65, 0x66, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x38, 0x0a,\n\t0x0b, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x72, 0x69, 0x67,\n\t0x68, 0x74, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x45, 0x0a, 0x0b, 0x73, 0x65, 0x74, 0x5f, 0x6f,\n\t0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x65, 0x74,\n\t0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x54,\n\t0x79, 0x70, 0x65, 0x52, 0x09, 0x73, 0x65, 0x74, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a,\n\t0x0a, 0x06, 0x69, 0x73, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00,\n\t0x52, 0x05, 0x69, 0x73, 0x41, 0x6c, 0x6c, 0x88, 0x01, 0x01, 0x12, 0x1c, 0x0a, 0x07, 0x62, 0x79,\n\t0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x48, 0x01, 0x52, 0x06, 0x62,\n\t0x79, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12, 0x37, 0x0a, 0x15, 0x61, 0x6c, 0x6c, 0x6f,\n\t0x77, 0x5f, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x48, 0x02, 0x52, 0x13, 0x61, 0x6c, 0x6c, 0x6f, 0x77,\n\t0x4d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x88, 0x01,\n\t0x01, 0x22, 0x72, 0x0a, 0x09, 0x53, 0x65, 0x74, 0x4f, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b,\n\t0x0a, 0x17, 0x53, 0x45, 0x54, 0x5f, 0x4f, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e,\n\t0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x53,\n\t0x45, 0x54, 0x5f, 0x4f, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x49, 0x4e, 0x54, 0x45, 0x52,\n\t0x53, 0x45, 0x43, 0x54, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x45, 0x54, 0x5f, 0x4f, 0x50,\n\t0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x4f, 0x4e, 0x10, 0x02, 0x12, 0x16, 0x0a,\n\t0x12, 0x53, 0x45, 0x54, 0x5f, 0x4f, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x58, 0x43,\n\t0x45, 0x50, 0x54, 0x10, 0x03, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x69, 0x73, 0x5f, 0x61, 0x6c, 0x6c,\n\t0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x62, 0x79, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x18, 0x0a, 0x16,\n\t0x5f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x63,\n\t0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0x4c, 0x0a, 0x05, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12,\n\t0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x14,\n\t0x0a, 0x05, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6c,\n\t0x69, 0x6d, 0x69, 0x74, 0x22, 0x4f, 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x2d,\n\t0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x16, 0x0a,\n\t0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x6f,\n\t0x66, 0x66, 0x73, 0x65, 0x74, 0x22, 0x4b, 0x0a, 0x04, 0x54, 0x61, 0x69, 0x6c, 0x12, 0x2d, 0x0a,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x14, 0x0a, 0x05,\n\t0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6c, 0x69, 0x6d,\n\t0x69, 0x74, 0x22, 0xfe, 0x05, 0x0a, 0x09, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x41, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x47, 0x72,\n\t0x6f, 0x75, 0x70, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x54, 0x79,\n\t0x70, 0x65, 0x12, 0x4c, 0x0a, 0x14, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x65,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b,\n\t0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x67, 0x72, 0x6f,\n\t0x75, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73,\n\t0x12, 0x4e, 0x0a, 0x15, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x65, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x14, 0x61, 0x67, 0x67, 0x72,\n\t0x65, 0x67, 0x61, 0x74, 0x65, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73,\n\t0x12, 0x34, 0x0a, 0x05, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x50, 0x69, 0x76, 0x6f, 0x74, 0x52,\n\t0x05, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x12, 0x4a, 0x0a, 0x0d, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69,\n\t0x6e, 0x67, 0x5f, 0x73, 0x65, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x41, 0x67,\n\t0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67,\n\t0x53, 0x65, 0x74, 0x73, 0x52, 0x0c, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x53, 0x65,\n\t0x74, 0x73, 0x1a, 0x6f, 0x0a, 0x05, 0x50, 0x69, 0x76, 0x6f, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x63,\n\t0x6f, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x52, 0x03, 0x63, 0x6f, 0x6c, 0x12, 0x39, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75,\n\t0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x06, 0x76, 0x61, 0x6c,\n\t0x75, 0x65, 0x73, 0x1a, 0x4c, 0x0a, 0x0c, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x53,\n\t0x65, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x0c, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x5f,\n\t0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x53, 0x65,\n\t0x74, 0x22, 0x9f, 0x01, 0x0a, 0x09, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12,\n\t0x1a, 0x0a, 0x16, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e,\n\t0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x47,\n\t0x52, 0x4f, 0x55, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x42,\n\t0x59, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x54, 0x59, 0x50,\n\t0x45, 0x5f, 0x52, 0x4f, 0x4c, 0x4c, 0x55, 0x50, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x47, 0x52,\n\t0x4f, 0x55, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x55, 0x42, 0x45, 0x10, 0x03, 0x12,\n\t0x14, 0x0a, 0x10, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x50, 0x49,\n\t0x56, 0x4f, 0x54, 0x10, 0x04, 0x12, 0x1c, 0x0a, 0x18, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x54,\n\t0x59, 0x50, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x45, 0x54,\n\t0x53, 0x10, 0x05, 0x22, 0xa0, 0x01, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x2d, 0x0a, 0x05,\n\t0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x39, 0x0a, 0x05, 0x6f,\n\t0x72, 0x64, 0x65, 0x72, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x52,\n\t0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x67, 0x6c, 0x6f,\n\t0x62, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x08, 0x69, 0x73, 0x47,\n\t0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x88, 0x01, 0x01, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x69, 0x73, 0x5f,\n\t0x67, 0x6c, 0x6f, 0x62, 0x61, 0x6c, 0x22, 0x8d, 0x01, 0x0a, 0x04, 0x44, 0x72, 0x6f, 0x70, 0x12,\n\t0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x33,\n\t0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0xf0, 0x01, 0x0a, 0x0b, 0x44, 0x65, 0x64, 0x75, 0x70,\n\t0x6c, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05,\n\t0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c,\n\t0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x13, 0x61, 0x6c, 0x6c, 0x5f,\n\t0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x61, 0x73, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18,\n\t0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x10, 0x61, 0x6c, 0x6c, 0x43, 0x6f, 0x6c, 0x75,\n\t0x6d, 0x6e, 0x73, 0x41, 0x73, 0x4b, 0x65, 0x79, 0x73, 0x88, 0x01, 0x01, 0x12, 0x2e, 0x0a, 0x10,\n\t0x77, 0x69, 0x74, 0x68, 0x69, 0x6e, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b,\n\t0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x01, 0x52, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x69, 0x6e,\n\t0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x88, 0x01, 0x01, 0x42, 0x16, 0x0a, 0x14,\n\t0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x61, 0x73, 0x5f,\n\t0x6b, 0x65, 0x79, 0x73, 0x42, 0x13, 0x0a, 0x11, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x69, 0x6e, 0x5f,\n\t0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x22, 0x59, 0x0a, 0x0d, 0x4c, 0x6f, 0x63,\n\t0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x17, 0x0a, 0x04, 0x64, 0x61,\n\t0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61,\n\t0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01,\n\t0x42, 0x07, 0x0a, 0x05, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x73, 0x63,\n\t0x68, 0x65, 0x6d, 0x61, 0x22, 0x48, 0x0a, 0x13, 0x43, 0x61, 0x63, 0x68, 0x65, 0x64, 0x4c, 0x6f,\n\t0x63, 0x61, 0x6c, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x68,\n\t0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x4a,\n\t0x04, 0x08, 0x01, 0x10, 0x02, 0x4a, 0x04, 0x08, 0x02, 0x10, 0x03, 0x52, 0x06, 0x75, 0x73, 0x65,\n\t0x72, 0x49, 0x64, 0x52, 0x09, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x37,\n\t0x0a, 0x14, 0x43, 0x61, 0x63, 0x68, 0x65, 0x64, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x91, 0x02, 0x0a, 0x06, 0x53, 0x61, 0x6d, 0x70,\n\t0x6c, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75,\n\t0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x6f, 0x75,\n\t0x6e, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x75, 0x70, 0x70, 0x65, 0x72, 0x5f, 0x62, 0x6f, 0x75, 0x6e,\n\t0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x75, 0x70, 0x70, 0x65, 0x72, 0x42, 0x6f,\n\t0x75, 0x6e, 0x64, 0x12, 0x2e, 0x0a, 0x10, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x72, 0x65, 0x70, 0x6c,\n\t0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52,\n\t0x0f, 0x77, 0x69, 0x74, 0x68, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74,\n\t0x88, 0x01, 0x01, 0x12, 0x17, 0x0a, 0x04, 0x73, 0x65, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28,\n\t0x03, 0x48, 0x01, 0x52, 0x04, 0x73, 0x65, 0x65, 0x64, 0x88, 0x01, 0x01, 0x12, 0x2f, 0x0a, 0x13,\n\t0x64, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x69, 0x73, 0x74, 0x69, 0x63, 0x5f, 0x6f, 0x72,\n\t0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x74, 0x65, 0x72,\n\t0x6d, 0x69, 0x6e, 0x69, 0x73, 0x74, 0x69, 0x63, 0x4f, 0x72, 0x64, 0x65, 0x72, 0x42, 0x13, 0x0a,\n\t0x11, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65,\n\t0x6e, 0x74, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x73, 0x65, 0x65, 0x64, 0x22, 0x91, 0x01, 0x0a, 0x05,\n\t0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x19, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x88, 0x01, 0x01,\n\t0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x65,\n\t0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x74, 0x65, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,\n\t0x52, 0x04, 0x73, 0x74, 0x65, 0x70, 0x12, 0x2a, 0x0a, 0x0e, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61,\n\t0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01,\n\t0x52, 0x0d, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x88,\n\t0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x42, 0x11, 0x0a, 0x0f,\n\t0x5f, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22,\n\t0x72, 0x0a, 0x0d, 0x53, 0x75, 0x62, 0x71, 0x75, 0x65, 0x72, 0x79, 0x41, 0x6c, 0x69, 0x61, 0x73,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x14, 0x0a, 0x05, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,\n\t0x61, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x66, 0x69,\n\t0x65, 0x72, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x71, 0x75, 0x61, 0x6c, 0x69, 0x66,\n\t0x69, 0x65, 0x72, 0x22, 0x8e, 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,\n\t0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70,\n\t0x75, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,\n\t0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x6e, 0x75, 0x6d, 0x50,\n\t0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x73, 0x68, 0x75,\n\t0x66, 0x66, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x07, 0x73, 0x68,\n\t0x75, 0x66, 0x66, 0x6c, 0x65, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x73, 0x68, 0x75,\n\t0x66, 0x66, 0x6c, 0x65, 0x22, 0x8e, 0x01, 0x0a, 0x0a, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x74, 0x72,\n\t0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70,\n\t0x75, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1a, 0x0a,\n\t0x08, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52,\n\t0x08, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x76, 0x65, 0x72,\n\t0x74, 0x69, 0x63, 0x61, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x76, 0x65, 0x72,\n\t0x74, 0x69, 0x63, 0x61, 0x6c, 0x22, 0x72, 0x0a, 0x0a, 0x48, 0x74, 0x6d, 0x6c, 0x53, 0x74, 0x72,\n\t0x69, 0x6e, 0x67, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70,\n\t0x75, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1a, 0x0a,\n\t0x08, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52,\n\t0x08, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x22, 0x5c, 0x0a, 0x0b, 0x53, 0x74, 0x61,\n\t0x74, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75,\n\t0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x69,\n\t0x73, 0x74, 0x69, 0x63, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61,\n\t0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x22, 0x51, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x44,\n\t0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x18, 0x02,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x22, 0x65, 0x0a, 0x0c, 0x53, 0x74,\n\t0x61, 0x74, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x74, 0x61, 0x62, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c,\n\t0x31, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x31, 0x12, 0x12, 0x0a,\n\t0x04, 0x63, 0x6f, 0x6c, 0x32, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c,\n\t0x32, 0x22, 0x60, 0x0a, 0x07, 0x53, 0x74, 0x61, 0x74, 0x43, 0x6f, 0x76, 0x12, 0x2d, 0x0a, 0x05,\n\t0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63,\n\t0x6f, 0x6c, 0x31, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x31, 0x12,\n\t0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x32, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63,\n\t0x6f, 0x6c, 0x32, 0x22, 0x89, 0x01, 0x0a, 0x08, 0x53, 0x74, 0x61, 0x74, 0x43, 0x6f, 0x72, 0x72,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x31, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63,\n\t0x6f, 0x6c, 0x31, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x32, 0x18, 0x03, 0x20, 0x01, 0x28,\n\t0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x32, 0x12, 0x1b, 0x0a, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f,\n\t0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f,\n\t0x64, 0x88, 0x01, 0x01, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x22,\n\t0xa4, 0x01, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x74, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x51, 0x75,\n\t0x61, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05,\n\t0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20,\n\t0x03, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x70, 0x72, 0x6f,\n\t0x62, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x01,\n\t0x52, 0x0d, 0x70, 0x72, 0x6f, 0x62, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12,\n\t0x25, 0x0a, 0x0e, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x65, 0x72, 0x72, 0x6f,\n\t0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0d, 0x72, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x76,\n\t0x65, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x7d, 0x0a, 0x0d, 0x53, 0x74, 0x61, 0x74, 0x46, 0x72,\n\t0x65, 0x71, 0x49, 0x74, 0x65, 0x6d, 0x73, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x18, 0x02,\n\t0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x73, 0x75,\n\t0x70, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x07, 0x73,\n\t0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x88, 0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x73, 0x75,\n\t0x70, 0x70, 0x6f, 0x72, 0x74, 0x22, 0xb5, 0x02, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x53, 0x61,\n\t0x6d, 0x70, 0x6c, 0x65, 0x42, 0x79, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05,\n\t0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x63, 0x6f, 0x6c, 0x18, 0x02, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x63,\n\t0x6f, 0x6c, 0x12, 0x42, 0x0a, 0x09, 0x66, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18,\n\t0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65,\n\t0x42, 0x79, 0x2e, 0x46, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x66, 0x72, 0x61,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x17, 0x0a, 0x04, 0x73, 0x65, 0x65, 0x64, 0x18, 0x05,\n\t0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x04, 0x73, 0x65, 0x65, 0x64, 0x88, 0x01, 0x01, 0x1a,\n\t0x63, 0x0a, 0x08, 0x46, 0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x07, 0x73,\n\t0x74, 0x72, 0x61, 0x74, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52,\n\t0x07, 0x73, 0x74, 0x72, 0x61, 0x74, 0x75, 0x6d, 0x12, 0x1a, 0x0a, 0x08, 0x66, 0x72, 0x61, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x08, 0x66, 0x72, 0x61, 0x63,\n\t0x74, 0x69, 0x6f, 0x6e, 0x42, 0x07, 0x0a, 0x05, 0x5f, 0x73, 0x65, 0x65, 0x64, 0x22, 0x86, 0x01,\n\t0x0a, 0x06, 0x4e, 0x41, 0x46, 0x69, 0x6c, 0x6c, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75,\n\t0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x18,\n\t0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x12, 0x39, 0x0a, 0x06, 0x76,\n\t0x61, 0x6c, 0x75, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x06,\n\t0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x86, 0x01, 0x0a, 0x06, 0x4e, 0x41, 0x44, 0x72, 0x6f,\n\t0x70, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74,\n\t0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04,\n\t0x63, 0x6f, 0x6c, 0x73, 0x12, 0x27, 0x0a, 0x0d, 0x6d, 0x69, 0x6e, 0x5f, 0x6e, 0x6f, 0x6e, 0x5f,\n\t0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0b, 0x6d,\n\t0x69, 0x6e, 0x4e, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x73, 0x88, 0x01, 0x01, 0x42, 0x10, 0x0a,\n\t0x0e, 0x5f, 0x6d, 0x69, 0x6e, 0x5f, 0x6e, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x22,\n\t0xa8, 0x02, 0x0a, 0x09, 0x4e, 0x41, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x12, 0x2d, 0x0a,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04,\n\t0x63, 0x6f, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x6c, 0x73,\n\t0x12, 0x48, 0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73,\n\t0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4e, 0x41, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65,\n\t0x2e, 0x52, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0c, 0x72, 0x65,\n\t0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x8d, 0x01, 0x0a, 0x0b, 0x52,\n\t0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x3e, 0x0a, 0x09, 0x6f, 0x6c,\n\t0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x52, 0x08, 0x6f, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x3e, 0x0a, 0x09, 0x6e, 0x65,\n\t0x77, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c,\n\t0x52, 0x08, 0x6e, 0x65, 0x77, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x58, 0x0a, 0x04, 0x54, 0x6f,\n\t0x44, 0x46, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75,\n\t0x74, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65,\n\t0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e,\n\t0x61, 0x6d, 0x65, 0x73, 0x22, 0xfe, 0x02, 0x0a, 0x12, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c,\n\t0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x12, 0x2d, 0x0a, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x69, 0x0a, 0x12, 0x72, 0x65,\n\t0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x6d, 0x61, 0x70,\n\t0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65,\n\t0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x42,\n\t0x02, 0x18, 0x01, 0x52, 0x10, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x4d, 0x61, 0x70, 0x12, 0x42, 0x0a, 0x07, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x73,\n\t0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x64, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65,\n\t0x52, 0x07, 0x72, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x43, 0x0a, 0x15, 0x52, 0x65, 0x6e,\n\t0x61, 0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x4d, 0x61, 0x70, 0x45, 0x6e, 0x74,\n\t0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x45,\n\t0x0a, 0x06, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x6c, 0x5f,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x4e,\n\t0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0c, 0x6e, 0x65, 0x77, 0x5f, 0x63, 0x6f, 0x6c, 0x5f, 0x6e,\n\t0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6e, 0x65, 0x77, 0x43, 0x6f,\n\t0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x77, 0x0a, 0x0b, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6c,\n\t0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x12, 0x39, 0x0a, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x18, 0x02,\n\t0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x2e,\n\t0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x07, 0x61, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x22, 0x86,\n\t0x01, 0x0a, 0x0d, 0x57, 0x69, 0x74, 0x68, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x1d, 0x0a, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x27,\n\t0x0a, 0x0f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x5f, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c,\n\t0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x54, 0x68,\n\t0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x22, 0x84, 0x01, 0x0a, 0x04, 0x48, 0x69, 0x6e, 0x74,\n\t0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12,\n\t0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,\n\t0x61, 0x6d, 0x65, 0x12, 0x39, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72,\n\t0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x22, 0xc7,\n\t0x02, 0x0a, 0x07, 0x55, 0x6e, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x2b, 0x0a, 0x03, 0x69, 0x64, 0x73,\n\t0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x03, 0x69, 0x64, 0x73, 0x12, 0x3a, 0x0a, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x55, 0x6e, 0x70, 0x69, 0x76, 0x6f, 0x74, 0x2e, 0x56,\n\t0x61, 0x6c, 0x75, 0x65, 0x73, 0x48, 0x00, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x88,\n\t0x01, 0x01, 0x12, 0x30, 0x0a, 0x14, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x63,\n\t0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,\n\t0x52, 0x12, 0x76, 0x61, 0x72, 0x69, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,\n\t0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2a, 0x0a, 0x11, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x63, 0x6f,\n\t0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x0f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65,\n\t0x1a, 0x3b, 0x0a, 0x06, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x12, 0x31, 0x0a, 0x06, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x42, 0x09, 0x0a,\n\t0x07, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x73, 0x22, 0x7a, 0x0a, 0x09, 0x54, 0x72, 0x61, 0x6e,\n\t0x73, 0x70, 0x6f, 0x73, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x12, 0x3e, 0x0a, 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x63, 0x6f,\n\t0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72,\n\t0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x6f, 0x6c,\n\t0x75, 0x6d, 0x6e, 0x73, 0x22, 0x7d, 0x0a, 0x1d, 0x55, 0x6e, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76,\n\t0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x64, 0x46, 0x75, 0x6e,\n\t0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75,\n\t0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x09, 0x61, 0x72,\n\t0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65,\n\t0x6e, 0x74, 0x73, 0x22, 0x6a, 0x0a, 0x08, 0x54, 0x6f, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12,\n\t0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x2f,\n\t0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22,\n\t0xcb, 0x01, 0x0a, 0x17, 0x52, 0x65, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x42,\n\t0x79, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x0a, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42, 0x0a, 0x0f, 0x70, 0x61,\n\t0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x73, 0x18, 0x02, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0e,\n\t0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x70, 0x72, 0x73, 0x12, 0x2a,\n\t0x0a, 0x0e, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0d, 0x6e, 0x75, 0x6d, 0x50, 0x61, 0x72,\n\t0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x88, 0x01, 0x01, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6e,\n\t0x75, 0x6d, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xe8, 0x01,\n\t0x0a, 0x0d, 0x4d, 0x61, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12,\n\t0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x42,\n\t0x0a, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66,\n\t0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x66, 0x75,\n\t0x6e, 0x63, 0x12, 0x22, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x62, 0x61, 0x72, 0x72, 0x69, 0x65, 0x72,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x69, 0x73, 0x42, 0x61, 0x72, 0x72,\n\t0x69, 0x65, 0x72, 0x88, 0x01, 0x01, 0x12, 0x22, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x66, 0x69, 0x6c,\n\t0x65, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x09, 0x70, 0x72,\n\t0x6f, 0x66, 0x69, 0x6c, 0x65, 0x49, 0x64, 0x88, 0x01, 0x01, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x69,\n\t0x73, 0x5f, 0x62, 0x61, 0x72, 0x72, 0x69, 0x65, 0x72, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x70, 0x72,\n\t0x6f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x22, 0xd2, 0x06, 0x0a, 0x08, 0x47, 0x72, 0x6f,\n\t0x75, 0x70, 0x4d, 0x61, 0x70, 0x12, 0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x12, 0x4c, 0x0a, 0x14, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67,\n\t0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03,\n\t0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x13, 0x67,\n\t0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x73, 0x12, 0x42, 0x0a, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65,\n\t0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e,\n\t0x52, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x12, 0x4a, 0x0a, 0x13, 0x73, 0x6f, 0x72, 0x74, 0x69, 0x6e,\n\t0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20,\n\t0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x12,\n\t0x73, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x73, 0x12, 0x3c, 0x0a, 0x0d, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x0c, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x70, 0x75, 0x74,\n\t0x12, 0x5b, 0x0a, 0x1c, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x67, 0x72, 0x6f, 0x75,\n\t0x70, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73,\n\t0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x1a, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x69,\n\t0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3b, 0x0a,\n\t0x18, 0x69, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x5f, 0x77,\n\t0x69, 0x74, 0x68, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x48,\n\t0x00, 0x52, 0x14, 0x69, 0x73, 0x4d, 0x61, 0x70, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x57, 0x69,\n\t0x74, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x88, 0x01, 0x01, 0x12, 0x24, 0x0a, 0x0b, 0x6f, 0x75,\n\t0x74, 0x70, 0x75, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x01, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x4d, 0x6f, 0x64, 0x65, 0x88, 0x01, 0x01,\n\t0x12, 0x26, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x66,\n\t0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75,\n\t0x74, 0x43, 0x6f, 0x6e, 0x66, 0x88, 0x01, 0x01, 0x12, 0x3f, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x74,\n\t0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x03, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x65,\n\t0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x65, 0x0a, 0x19, 0x74, 0x72, 0x61,\n\t0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x74, 0x61, 0x74,\n\t0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x54, 0x72, 0x61,\n\t0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49,\n\t0x6e, 0x66, 0x6f, 0x48, 0x04, 0x52, 0x16, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d,\n\t0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x88, 0x01, 0x01,\n\t0x42, 0x1b, 0x0a, 0x19, 0x5f, 0x69, 0x73, 0x5f, 0x6d, 0x61, 0x70, 0x5f, 0x67, 0x72, 0x6f, 0x75,\n\t0x70, 0x73, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x42, 0x0e, 0x0a,\n\t0x0c, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x42, 0x0f, 0x0a,\n\t0x0d, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x42, 0x0f,\n\t0x0a, 0x0d, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x42,\n\t0x1c, 0x0a, 0x1a, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x77, 0x69,\n\t0x74, 0x68, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0xdf, 0x01,\n\t0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x57, 0x69, 0x74, 0x68, 0x53,\n\t0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65,\n\t0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x69, 0x6d,\n\t0x65, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x38, 0x0a, 0x16, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74,\n\t0x69, 0x6d, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x13, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69,\n\t0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x88, 0x01, 0x01, 0x12,\n\t0x41, 0x0a, 0x0d, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48,\n\t0x01, 0x52, 0x0c, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88,\n\t0x01, 0x01, 0x42, 0x19, 0x0a, 0x17, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d,\n\t0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x42, 0x10, 0x0a,\n\t0x0e, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22,\n\t0x8e, 0x04, 0x0a, 0x0a, 0x43, 0x6f, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4d, 0x61, 0x70, 0x12, 0x2d,\n\t0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65,\n\t0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x57, 0x0a,\n\t0x1a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x5f,\n\t0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x69, 0x6e,\n\t0x70, 0x75, 0x74, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2d, 0x0a, 0x05, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x18,\n\t0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05,\n\t0x6f, 0x74, 0x68, 0x65, 0x72, 0x12, 0x57, 0x0a, 0x1a, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x5f, 0x67,\n\t0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69,\n\t0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x18, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70,\n\t0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x42,\n\t0x0a, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66,\n\t0x69, 0x6e, 0x65, 0x64, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x66, 0x75,\n\t0x6e, 0x63, 0x12, 0x55, 0x0a, 0x19, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x6f, 0x72, 0x74,\n\t0x69, 0x6e, 0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18,\n\t0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,\n\t0x52, 0x17, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x45, 0x78,\n\t0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x55, 0x0a, 0x19, 0x6f, 0x74, 0x68,\n\t0x65, 0x72, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65,\n\t0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x17, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x53, 0x6f,\n\t0x72, 0x74, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73,\n\t0x22, 0xe5, 0x02, 0x0a, 0x16, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x49, 0x6e, 0x50, 0x61, 0x6e, 0x64,\n\t0x61, 0x73, 0x57, 0x69, 0x74, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x69,\n\t0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x4c, 0x0a, 0x14, 0x67, 0x72,\n\t0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73,\n\t0x69, 0x6f, 0x6e, 0x52, 0x13, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x69, 0x6e, 0x67, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x42, 0x0a, 0x04, 0x66, 0x75, 0x6e, 0x63,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c,\n\t0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x46, 0x75,\n\t0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x66, 0x75, 0x6e, 0x63, 0x12, 0x23, 0x0a, 0x0d,\n\t0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x04, 0x20,\n\t0x01, 0x28, 0x09, 0x52, 0x0c, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d,\n\t0x61, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d,\n\t0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x65, 0x53, 0x63,\n\t0x68, 0x65, 0x6d, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x6d,\n\t0x6f, 0x64, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6f, 0x75, 0x74, 0x70, 0x75,\n\t0x74, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74,\n\t0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x69, 0x6d,\n\t0x65, 0x6f, 0x75, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x22, 0xf4, 0x01, 0x0a, 0x24, 0x43, 0x6f, 0x6d,\n\t0x6d, 0x6f, 0x6e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66,\n\t0x69, 0x6e, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,\n\t0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61,\n\t0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69,\n\t0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x64, 0x65, 0x74, 0x65, 0x72, 0x6d,\n\t0x69, 0x6e, 0x69, 0x73, 0x74, 0x69, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x64,\n\t0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x69, 0x73, 0x74, 0x69, 0x63, 0x12, 0x37, 0x0a, 0x09,\n\t0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x61, 0x72, 0x67, 0x75,\n\t0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x3c, 0x0a, 0x0b, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f,\n\t0x75, 0x64, 0x74, 0x66, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74, 0x68, 0x6f,\n\t0x6e, 0x55, 0x44, 0x54, 0x46, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x55,\n\t0x64, 0x74, 0x66, 0x42, 0x0a, 0x0a, 0x08, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22,\n\t0xb1, 0x01, 0x0a, 0x0a, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x55, 0x44, 0x54, 0x46, 0x12, 0x3d,\n\t0x0a, 0x0b, 0x72, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a,\n\t0x72, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a,\n\t0x09, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05,\n\t0x52, 0x08, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f,\n\t0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, 0x6f, 0x6d,\n\t0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x76,\n\t0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e,\n\t0x56, 0x65, 0x72, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x72, 0x65, 0x74, 0x75, 0x72, 0x6e, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x22, 0x97, 0x01, 0x0a, 0x21, 0x43, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x49, 0x6e,\n\t0x6c, 0x69, 0x6e, 0x65, 0x55, 0x73, 0x65, 0x72, 0x44, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x44,\n\t0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d,\n\t0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x4f, 0x0a,\n\t0x12, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f, 0x75,\n\t0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e,\n\t0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x48, 0x00, 0x52, 0x10, 0x70, 0x79,\n\t0x74, 0x68, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x42, 0x0d,\n\t0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0x4b, 0x0a,\n\t0x10, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6f, 0x75, 0x72, 0x63,\n\t0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0c, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70,\n\t0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x09, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x56, 0x65, 0x72, 0x22, 0x88, 0x01, 0x0a, 0x0e, 0x43,\n\t0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x0a,\n\t0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x12, 0x0a, 0x04,\n\t0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,\n\t0x12, 0x33, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28,\n\t0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x6d, 0x65,\n\t0x74, 0x72, 0x69, 0x63, 0x73, 0x22, 0x84, 0x03, 0x0a, 0x05, 0x50, 0x61, 0x72, 0x73, 0x65, 0x12,\n\t0x2d, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52,\n\t0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x38,\n\t0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x50,\n\t0x61, 0x72, 0x73, 0x65, 0x2e, 0x50, 0x61, 0x72, 0x73, 0x65, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74,\n\t0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x34, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65,\n\t0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70,\n\t0x65, 0x48, 0x00, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x88, 0x01, 0x01, 0x12, 0x3b,\n\t0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32,\n\t0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x50, 0x61, 0x72, 0x73, 0x65, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74,\n\t0x72, 0x79, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f,\n\t0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,\n\t0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,\n\t0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,\n\t0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x58, 0x0a, 0x0b, 0x50, 0x61, 0x72, 0x73, 0x65,\n\t0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x1c, 0x0a, 0x18, 0x50, 0x41, 0x52, 0x53, 0x45, 0x5f,\n\t0x46, 0x4f, 0x52, 0x4d, 0x41, 0x54, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49,\n\t0x45, 0x44, 0x10, 0x00, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x41, 0x52, 0x53, 0x45, 0x5f, 0x46, 0x4f,\n\t0x52, 0x4d, 0x41, 0x54, 0x5f, 0x43, 0x53, 0x56, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x50, 0x41,\n\t0x52, 0x53, 0x45, 0x5f, 0x46, 0x4f, 0x52, 0x4d, 0x41, 0x54, 0x5f, 0x4a, 0x53, 0x4f, 0x4e, 0x10,\n\t0x02, 0x42, 0x09, 0x0a, 0x07, 0x5f, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xdb, 0x03, 0x0a,\n\t0x08, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65, 0x66,\n\t0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x52, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05,\n\t0x72, 0x69, 0x67, 0x68, 0x74, 0x12, 0x37, 0x0a, 0x0a, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x61, 0x73,\n\t0x5f, 0x6f, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73,\n\t0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x65, 0x66, 0x74, 0x41, 0x73, 0x4f, 0x66, 0x12, 0x39,\n\t0x0a, 0x0b, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x61, 0x73, 0x5f, 0x6f, 0x66, 0x18, 0x04, 0x20,\n\t0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,\n\t0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x09,\n\t0x72, 0x69, 0x67, 0x68, 0x74, 0x41, 0x73, 0x4f, 0x66, 0x12, 0x36, 0x0a, 0x09, 0x6a, 0x6f, 0x69,\n\t0x6e, 0x5f, 0x65, 0x78, 0x70, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70,\n\t0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6a, 0x6f, 0x69, 0x6e, 0x45, 0x78, 0x70,\n\t0x72, 0x12, 0x23, 0x0a, 0x0d, 0x75, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d,\n\t0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x75, 0x73, 0x69, 0x6e, 0x67, 0x43,\n\t0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6a, 0x6f, 0x69, 0x6e, 0x54,\n\t0x79, 0x70, 0x65, 0x12, 0x37, 0x0a, 0x09, 0x74, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x6e, 0x63, 0x65,\n\t0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f,\n\t0x6e, 0x52, 0x09, 0x74, 0x6f, 0x6c, 0x65, 0x72, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x2e, 0x0a, 0x13,\n\t0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63,\n\t0x68, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x11, 0x61, 0x6c, 0x6c, 0x6f, 0x77,\n\t0x45, 0x78, 0x61, 0x63, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09,\n\t0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52,\n\t0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xe6, 0x01, 0x0a, 0x0b, 0x4c,\n\t0x61, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x2b, 0x0a, 0x04, 0x6c, 0x65,\n\t0x66, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x52, 0x04, 0x6c, 0x65, 0x66, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x72, 0x69, 0x67, 0x68, 0x74,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x52, 0x65, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,\n\t0x05, 0x72, 0x69, 0x67, 0x68, 0x74, 0x12, 0x40, 0x0a, 0x0e, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x63,\n\t0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x45,\n\t0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x6a, 0x6f, 0x69, 0x6e, 0x43,\n\t0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x09, 0x6a, 0x6f, 0x69, 0x6e,\n\t0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x4a, 0x6f, 0x69, 0x6e,\n\t0x2e, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6a, 0x6f, 0x69, 0x6e, 0x54,\n\t0x79, 0x70, 0x65, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,\n\t0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,\n\t0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06, 0x70, 0x72, 0x6f,\n\t0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_relations_proto_rawDescOnce sync.Once\n\tfile_spark_connect_relations_proto_rawDescData = file_spark_connect_relations_proto_rawDesc\n)\n\nfunc file_spark_connect_relations_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_relations_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_relations_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_relations_proto_rawDescData)\n\t})\n\treturn file_spark_connect_relations_proto_rawDescData\n}\n\nvar file_spark_connect_relations_proto_enumTypes = make([]protoimpl.EnumInfo, 4)\nvar file_spark_connect_relations_proto_msgTypes = make([]protoimpl.MessageInfo, 80)\nvar file_spark_connect_relations_proto_goTypes = []interface{}{\n\t(Join_JoinType)(0),                           // 0: spark.connect.Join.JoinType\n\t(SetOperation_SetOpType)(0),                  // 1: spark.connect.SetOperation.SetOpType\n\t(Aggregate_GroupType)(0),                     // 2: spark.connect.Aggregate.GroupType\n\t(Parse_ParseFormat)(0),                       // 3: spark.connect.Parse.ParseFormat\n\t(*Relation)(nil),                             // 4: spark.connect.Relation\n\t(*MlRelation)(nil),                           // 5: spark.connect.MlRelation\n\t(*Fetch)(nil),                                // 6: spark.connect.Fetch\n\t(*Unknown)(nil),                              // 7: spark.connect.Unknown\n\t(*RelationCommon)(nil),                       // 8: spark.connect.RelationCommon\n\t(*SQL)(nil),                                  // 9: spark.connect.SQL\n\t(*WithRelations)(nil),                        // 10: spark.connect.WithRelations\n\t(*Read)(nil),                                 // 11: spark.connect.Read\n\t(*Project)(nil),                              // 12: spark.connect.Project\n\t(*Filter)(nil),                               // 13: spark.connect.Filter\n\t(*Join)(nil),                                 // 14: spark.connect.Join\n\t(*SetOperation)(nil),                         // 15: spark.connect.SetOperation\n\t(*Limit)(nil),                                // 16: spark.connect.Limit\n\t(*Offset)(nil),                               // 17: spark.connect.Offset\n\t(*Tail)(nil),                                 // 18: spark.connect.Tail\n\t(*Aggregate)(nil),                            // 19: spark.connect.Aggregate\n\t(*Sort)(nil),                                 // 20: spark.connect.Sort\n\t(*Drop)(nil),                                 // 21: spark.connect.Drop\n\t(*Deduplicate)(nil),                          // 22: spark.connect.Deduplicate\n\t(*LocalRelation)(nil),                        // 23: spark.connect.LocalRelation\n\t(*CachedLocalRelation)(nil),                  // 24: spark.connect.CachedLocalRelation\n\t(*CachedRemoteRelation)(nil),                 // 25: spark.connect.CachedRemoteRelation\n\t(*Sample)(nil),                               // 26: spark.connect.Sample\n\t(*Range)(nil),                                // 27: spark.connect.Range\n\t(*SubqueryAlias)(nil),                        // 28: spark.connect.SubqueryAlias\n\t(*Repartition)(nil),                          // 29: spark.connect.Repartition\n\t(*ShowString)(nil),                           // 30: spark.connect.ShowString\n\t(*HtmlString)(nil),                           // 31: spark.connect.HtmlString\n\t(*StatSummary)(nil),                          // 32: spark.connect.StatSummary\n\t(*StatDescribe)(nil),                         // 33: spark.connect.StatDescribe\n\t(*StatCrosstab)(nil),                         // 34: spark.connect.StatCrosstab\n\t(*StatCov)(nil),                              // 35: spark.connect.StatCov\n\t(*StatCorr)(nil),                             // 36: spark.connect.StatCorr\n\t(*StatApproxQuantile)(nil),                   // 37: spark.connect.StatApproxQuantile\n\t(*StatFreqItems)(nil),                        // 38: spark.connect.StatFreqItems\n\t(*StatSampleBy)(nil),                         // 39: spark.connect.StatSampleBy\n\t(*NAFill)(nil),                               // 40: spark.connect.NAFill\n\t(*NADrop)(nil),                               // 41: spark.connect.NADrop\n\t(*NAReplace)(nil),                            // 42: spark.connect.NAReplace\n\t(*ToDF)(nil),                                 // 43: spark.connect.ToDF\n\t(*WithColumnsRenamed)(nil),                   // 44: spark.connect.WithColumnsRenamed\n\t(*WithColumns)(nil),                          // 45: spark.connect.WithColumns\n\t(*WithWatermark)(nil),                        // 46: spark.connect.WithWatermark\n\t(*Hint)(nil),                                 // 47: spark.connect.Hint\n\t(*Unpivot)(nil),                              // 48: spark.connect.Unpivot\n\t(*Transpose)(nil),                            // 49: spark.connect.Transpose\n\t(*UnresolvedTableValuedFunction)(nil),        // 50: spark.connect.UnresolvedTableValuedFunction\n\t(*ToSchema)(nil),                             // 51: spark.connect.ToSchema\n\t(*RepartitionByExpression)(nil),              // 52: spark.connect.RepartitionByExpression\n\t(*MapPartitions)(nil),                        // 53: spark.connect.MapPartitions\n\t(*GroupMap)(nil),                             // 54: spark.connect.GroupMap\n\t(*TransformWithStateInfo)(nil),               // 55: spark.connect.TransformWithStateInfo\n\t(*CoGroupMap)(nil),                           // 56: spark.connect.CoGroupMap\n\t(*ApplyInPandasWithState)(nil),               // 57: spark.connect.ApplyInPandasWithState\n\t(*CommonInlineUserDefinedTableFunction)(nil), // 58: spark.connect.CommonInlineUserDefinedTableFunction\n\t(*PythonUDTF)(nil),                           // 59: spark.connect.PythonUDTF\n\t(*CommonInlineUserDefinedDataSource)(nil),    // 60: spark.connect.CommonInlineUserDefinedDataSource\n\t(*PythonDataSource)(nil),                     // 61: spark.connect.PythonDataSource\n\t(*CollectMetrics)(nil),                       // 62: spark.connect.CollectMetrics\n\t(*Parse)(nil),                                // 63: spark.connect.Parse\n\t(*AsOfJoin)(nil),                             // 64: spark.connect.AsOfJoin\n\t(*LateralJoin)(nil),                          // 65: spark.connect.LateralJoin\n\t(*MlRelation_Transform)(nil),                 // 66: spark.connect.MlRelation.Transform\n\t(*Fetch_Method)(nil),                         // 67: spark.connect.Fetch.Method\n\t(*Fetch_Method_Args)(nil),                    // 68: spark.connect.Fetch.Method.Args\n\tnil,                                          // 69: spark.connect.SQL.ArgsEntry\n\tnil,                                          // 70: spark.connect.SQL.NamedArgumentsEntry\n\t(*Read_NamedTable)(nil),                      // 71: spark.connect.Read.NamedTable\n\t(*Read_DataSource)(nil),                      // 72: spark.connect.Read.DataSource\n\tnil,                                          // 73: spark.connect.Read.NamedTable.OptionsEntry\n\tnil,                                          // 74: spark.connect.Read.DataSource.OptionsEntry\n\t(*Join_JoinDataType)(nil),                    // 75: spark.connect.Join.JoinDataType\n\t(*Aggregate_Pivot)(nil),                      // 76: spark.connect.Aggregate.Pivot\n\t(*Aggregate_GroupingSets)(nil),               // 77: spark.connect.Aggregate.GroupingSets\n\t(*StatSampleBy_Fraction)(nil),                // 78: spark.connect.StatSampleBy.Fraction\n\t(*NAReplace_Replacement)(nil),                // 79: spark.connect.NAReplace.Replacement\n\tnil,                                          // 80: spark.connect.WithColumnsRenamed.RenameColumnsMapEntry\n\t(*WithColumnsRenamed_Rename)(nil),            // 81: spark.connect.WithColumnsRenamed.Rename\n\t(*Unpivot_Values)(nil),                       // 82: spark.connect.Unpivot.Values\n\tnil,                                          // 83: spark.connect.Parse.OptionsEntry\n\t(*Catalog)(nil),                              // 84: spark.connect.Catalog\n\t(*anypb.Any)(nil),                            // 85: google.protobuf.Any\n\t(*ObjectRef)(nil),                            // 86: spark.connect.ObjectRef\n\t(*Origin)(nil),                               // 87: spark.connect.Origin\n\t(*Expression_Literal)(nil),                   // 88: spark.connect.Expression.Literal\n\t(*Expression)(nil),                           // 89: spark.connect.Expression\n\t(*Expression_SortOrder)(nil),                 // 90: spark.connect.Expression.SortOrder\n\t(*Expression_Alias)(nil),                     // 91: spark.connect.Expression.Alias\n\t(*DataType)(nil),                             // 92: spark.connect.DataType\n\t(*CommonInlineUserDefinedFunction)(nil),      // 93: spark.connect.CommonInlineUserDefinedFunction\n\t(*MlOperator)(nil),                           // 94: spark.connect.MlOperator\n\t(*MlParams)(nil),                             // 95: spark.connect.MlParams\n}\nvar file_spark_connect_relations_proto_depIdxs = []int32{\n\t8,   // 0: spark.connect.Relation.common:type_name -> spark.connect.RelationCommon\n\t11,  // 1: spark.connect.Relation.read:type_name -> spark.connect.Read\n\t12,  // 2: spark.connect.Relation.project:type_name -> spark.connect.Project\n\t13,  // 3: spark.connect.Relation.filter:type_name -> spark.connect.Filter\n\t14,  // 4: spark.connect.Relation.join:type_name -> spark.connect.Join\n\t15,  // 5: spark.connect.Relation.set_op:type_name -> spark.connect.SetOperation\n\t20,  // 6: spark.connect.Relation.sort:type_name -> spark.connect.Sort\n\t16,  // 7: spark.connect.Relation.limit:type_name -> spark.connect.Limit\n\t19,  // 8: spark.connect.Relation.aggregate:type_name -> spark.connect.Aggregate\n\t9,   // 9: spark.connect.Relation.sql:type_name -> spark.connect.SQL\n\t23,  // 10: spark.connect.Relation.local_relation:type_name -> spark.connect.LocalRelation\n\t26,  // 11: spark.connect.Relation.sample:type_name -> spark.connect.Sample\n\t17,  // 12: spark.connect.Relation.offset:type_name -> spark.connect.Offset\n\t22,  // 13: spark.connect.Relation.deduplicate:type_name -> spark.connect.Deduplicate\n\t27,  // 14: spark.connect.Relation.range:type_name -> spark.connect.Range\n\t28,  // 15: spark.connect.Relation.subquery_alias:type_name -> spark.connect.SubqueryAlias\n\t29,  // 16: spark.connect.Relation.repartition:type_name -> spark.connect.Repartition\n\t43,  // 17: spark.connect.Relation.to_df:type_name -> spark.connect.ToDF\n\t44,  // 18: spark.connect.Relation.with_columns_renamed:type_name -> spark.connect.WithColumnsRenamed\n\t30,  // 19: spark.connect.Relation.show_string:type_name -> spark.connect.ShowString\n\t21,  // 20: spark.connect.Relation.drop:type_name -> spark.connect.Drop\n\t18,  // 21: spark.connect.Relation.tail:type_name -> spark.connect.Tail\n\t45,  // 22: spark.connect.Relation.with_columns:type_name -> spark.connect.WithColumns\n\t47,  // 23: spark.connect.Relation.hint:type_name -> spark.connect.Hint\n\t48,  // 24: spark.connect.Relation.unpivot:type_name -> spark.connect.Unpivot\n\t51,  // 25: spark.connect.Relation.to_schema:type_name -> spark.connect.ToSchema\n\t52,  // 26: spark.connect.Relation.repartition_by_expression:type_name -> spark.connect.RepartitionByExpression\n\t53,  // 27: spark.connect.Relation.map_partitions:type_name -> spark.connect.MapPartitions\n\t62,  // 28: spark.connect.Relation.collect_metrics:type_name -> spark.connect.CollectMetrics\n\t63,  // 29: spark.connect.Relation.parse:type_name -> spark.connect.Parse\n\t54,  // 30: spark.connect.Relation.group_map:type_name -> spark.connect.GroupMap\n\t56,  // 31: spark.connect.Relation.co_group_map:type_name -> spark.connect.CoGroupMap\n\t46,  // 32: spark.connect.Relation.with_watermark:type_name -> spark.connect.WithWatermark\n\t57,  // 33: spark.connect.Relation.apply_in_pandas_with_state:type_name -> spark.connect.ApplyInPandasWithState\n\t31,  // 34: spark.connect.Relation.html_string:type_name -> spark.connect.HtmlString\n\t24,  // 35: spark.connect.Relation.cached_local_relation:type_name -> spark.connect.CachedLocalRelation\n\t25,  // 36: spark.connect.Relation.cached_remote_relation:type_name -> spark.connect.CachedRemoteRelation\n\t58,  // 37: spark.connect.Relation.common_inline_user_defined_table_function:type_name -> spark.connect.CommonInlineUserDefinedTableFunction\n\t64,  // 38: spark.connect.Relation.as_of_join:type_name -> spark.connect.AsOfJoin\n\t60,  // 39: spark.connect.Relation.common_inline_user_defined_data_source:type_name -> spark.connect.CommonInlineUserDefinedDataSource\n\t10,  // 40: spark.connect.Relation.with_relations:type_name -> spark.connect.WithRelations\n\t49,  // 41: spark.connect.Relation.transpose:type_name -> spark.connect.Transpose\n\t50,  // 42: spark.connect.Relation.unresolved_table_valued_function:type_name -> spark.connect.UnresolvedTableValuedFunction\n\t65,  // 43: spark.connect.Relation.lateral_join:type_name -> spark.connect.LateralJoin\n\t40,  // 44: spark.connect.Relation.fill_na:type_name -> spark.connect.NAFill\n\t41,  // 45: spark.connect.Relation.drop_na:type_name -> spark.connect.NADrop\n\t42,  // 46: spark.connect.Relation.replace:type_name -> spark.connect.NAReplace\n\t32,  // 47: spark.connect.Relation.summary:type_name -> spark.connect.StatSummary\n\t34,  // 48: spark.connect.Relation.crosstab:type_name -> spark.connect.StatCrosstab\n\t33,  // 49: spark.connect.Relation.describe:type_name -> spark.connect.StatDescribe\n\t35,  // 50: spark.connect.Relation.cov:type_name -> spark.connect.StatCov\n\t36,  // 51: spark.connect.Relation.corr:type_name -> spark.connect.StatCorr\n\t37,  // 52: spark.connect.Relation.approx_quantile:type_name -> spark.connect.StatApproxQuantile\n\t38,  // 53: spark.connect.Relation.freq_items:type_name -> spark.connect.StatFreqItems\n\t39,  // 54: spark.connect.Relation.sample_by:type_name -> spark.connect.StatSampleBy\n\t84,  // 55: spark.connect.Relation.catalog:type_name -> spark.connect.Catalog\n\t5,   // 56: spark.connect.Relation.ml_relation:type_name -> spark.connect.MlRelation\n\t85,  // 57: spark.connect.Relation.extension:type_name -> google.protobuf.Any\n\t7,   // 58: spark.connect.Relation.unknown:type_name -> spark.connect.Unknown\n\t66,  // 59: spark.connect.MlRelation.transform:type_name -> spark.connect.MlRelation.Transform\n\t6,   // 60: spark.connect.MlRelation.fetch:type_name -> spark.connect.Fetch\n\t86,  // 61: spark.connect.Fetch.obj_ref:type_name -> spark.connect.ObjectRef\n\t67,  // 62: spark.connect.Fetch.methods:type_name -> spark.connect.Fetch.Method\n\t87,  // 63: spark.connect.RelationCommon.origin:type_name -> spark.connect.Origin\n\t69,  // 64: spark.connect.SQL.args:type_name -> spark.connect.SQL.ArgsEntry\n\t88,  // 65: spark.connect.SQL.pos_args:type_name -> spark.connect.Expression.Literal\n\t70,  // 66: spark.connect.SQL.named_arguments:type_name -> spark.connect.SQL.NamedArgumentsEntry\n\t89,  // 67: spark.connect.SQL.pos_arguments:type_name -> spark.connect.Expression\n\t4,   // 68: spark.connect.WithRelations.root:type_name -> spark.connect.Relation\n\t4,   // 69: spark.connect.WithRelations.references:type_name -> spark.connect.Relation\n\t71,  // 70: spark.connect.Read.named_table:type_name -> spark.connect.Read.NamedTable\n\t72,  // 71: spark.connect.Read.data_source:type_name -> spark.connect.Read.DataSource\n\t4,   // 72: spark.connect.Project.input:type_name -> spark.connect.Relation\n\t89,  // 73: spark.connect.Project.expressions:type_name -> spark.connect.Expression\n\t4,   // 74: spark.connect.Filter.input:type_name -> spark.connect.Relation\n\t89,  // 75: spark.connect.Filter.condition:type_name -> spark.connect.Expression\n\t4,   // 76: spark.connect.Join.left:type_name -> spark.connect.Relation\n\t4,   // 77: spark.connect.Join.right:type_name -> spark.connect.Relation\n\t89,  // 78: spark.connect.Join.join_condition:type_name -> spark.connect.Expression\n\t0,   // 79: spark.connect.Join.join_type:type_name -> spark.connect.Join.JoinType\n\t75,  // 80: spark.connect.Join.join_data_type:type_name -> spark.connect.Join.JoinDataType\n\t4,   // 81: spark.connect.SetOperation.left_input:type_name -> spark.connect.Relation\n\t4,   // 82: spark.connect.SetOperation.right_input:type_name -> spark.connect.Relation\n\t1,   // 83: spark.connect.SetOperation.set_op_type:type_name -> spark.connect.SetOperation.SetOpType\n\t4,   // 84: spark.connect.Limit.input:type_name -> spark.connect.Relation\n\t4,   // 85: spark.connect.Offset.input:type_name -> spark.connect.Relation\n\t4,   // 86: spark.connect.Tail.input:type_name -> spark.connect.Relation\n\t4,   // 87: spark.connect.Aggregate.input:type_name -> spark.connect.Relation\n\t2,   // 88: spark.connect.Aggregate.group_type:type_name -> spark.connect.Aggregate.GroupType\n\t89,  // 89: spark.connect.Aggregate.grouping_expressions:type_name -> spark.connect.Expression\n\t89,  // 90: spark.connect.Aggregate.aggregate_expressions:type_name -> spark.connect.Expression\n\t76,  // 91: spark.connect.Aggregate.pivot:type_name -> spark.connect.Aggregate.Pivot\n\t77,  // 92: spark.connect.Aggregate.grouping_sets:type_name -> spark.connect.Aggregate.GroupingSets\n\t4,   // 93: spark.connect.Sort.input:type_name -> spark.connect.Relation\n\t90,  // 94: spark.connect.Sort.order:type_name -> spark.connect.Expression.SortOrder\n\t4,   // 95: spark.connect.Drop.input:type_name -> spark.connect.Relation\n\t89,  // 96: spark.connect.Drop.columns:type_name -> spark.connect.Expression\n\t4,   // 97: spark.connect.Deduplicate.input:type_name -> spark.connect.Relation\n\t4,   // 98: spark.connect.Sample.input:type_name -> spark.connect.Relation\n\t4,   // 99: spark.connect.SubqueryAlias.input:type_name -> spark.connect.Relation\n\t4,   // 100: spark.connect.Repartition.input:type_name -> spark.connect.Relation\n\t4,   // 101: spark.connect.ShowString.input:type_name -> spark.connect.Relation\n\t4,   // 102: spark.connect.HtmlString.input:type_name -> spark.connect.Relation\n\t4,   // 103: spark.connect.StatSummary.input:type_name -> spark.connect.Relation\n\t4,   // 104: spark.connect.StatDescribe.input:type_name -> spark.connect.Relation\n\t4,   // 105: spark.connect.StatCrosstab.input:type_name -> spark.connect.Relation\n\t4,   // 106: spark.connect.StatCov.input:type_name -> spark.connect.Relation\n\t4,   // 107: spark.connect.StatCorr.input:type_name -> spark.connect.Relation\n\t4,   // 108: spark.connect.StatApproxQuantile.input:type_name -> spark.connect.Relation\n\t4,   // 109: spark.connect.StatFreqItems.input:type_name -> spark.connect.Relation\n\t4,   // 110: spark.connect.StatSampleBy.input:type_name -> spark.connect.Relation\n\t89,  // 111: spark.connect.StatSampleBy.col:type_name -> spark.connect.Expression\n\t78,  // 112: spark.connect.StatSampleBy.fractions:type_name -> spark.connect.StatSampleBy.Fraction\n\t4,   // 113: spark.connect.NAFill.input:type_name -> spark.connect.Relation\n\t88,  // 114: spark.connect.NAFill.values:type_name -> spark.connect.Expression.Literal\n\t4,   // 115: spark.connect.NADrop.input:type_name -> spark.connect.Relation\n\t4,   // 116: spark.connect.NAReplace.input:type_name -> spark.connect.Relation\n\t79,  // 117: spark.connect.NAReplace.replacements:type_name -> spark.connect.NAReplace.Replacement\n\t4,   // 118: spark.connect.ToDF.input:type_name -> spark.connect.Relation\n\t4,   // 119: spark.connect.WithColumnsRenamed.input:type_name -> spark.connect.Relation\n\t80,  // 120: spark.connect.WithColumnsRenamed.rename_columns_map:type_name -> spark.connect.WithColumnsRenamed.RenameColumnsMapEntry\n\t81,  // 121: spark.connect.WithColumnsRenamed.renames:type_name -> spark.connect.WithColumnsRenamed.Rename\n\t4,   // 122: spark.connect.WithColumns.input:type_name -> spark.connect.Relation\n\t91,  // 123: spark.connect.WithColumns.aliases:type_name -> spark.connect.Expression.Alias\n\t4,   // 124: spark.connect.WithWatermark.input:type_name -> spark.connect.Relation\n\t4,   // 125: spark.connect.Hint.input:type_name -> spark.connect.Relation\n\t89,  // 126: spark.connect.Hint.parameters:type_name -> spark.connect.Expression\n\t4,   // 127: spark.connect.Unpivot.input:type_name -> spark.connect.Relation\n\t89,  // 128: spark.connect.Unpivot.ids:type_name -> spark.connect.Expression\n\t82,  // 129: spark.connect.Unpivot.values:type_name -> spark.connect.Unpivot.Values\n\t4,   // 130: spark.connect.Transpose.input:type_name -> spark.connect.Relation\n\t89,  // 131: spark.connect.Transpose.index_columns:type_name -> spark.connect.Expression\n\t89,  // 132: spark.connect.UnresolvedTableValuedFunction.arguments:type_name -> spark.connect.Expression\n\t4,   // 133: spark.connect.ToSchema.input:type_name -> spark.connect.Relation\n\t92,  // 134: spark.connect.ToSchema.schema:type_name -> spark.connect.DataType\n\t4,   // 135: spark.connect.RepartitionByExpression.input:type_name -> spark.connect.Relation\n\t89,  // 136: spark.connect.RepartitionByExpression.partition_exprs:type_name -> spark.connect.Expression\n\t4,   // 137: spark.connect.MapPartitions.input:type_name -> spark.connect.Relation\n\t93,  // 138: spark.connect.MapPartitions.func:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t4,   // 139: spark.connect.GroupMap.input:type_name -> spark.connect.Relation\n\t89,  // 140: spark.connect.GroupMap.grouping_expressions:type_name -> spark.connect.Expression\n\t93,  // 141: spark.connect.GroupMap.func:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t89,  // 142: spark.connect.GroupMap.sorting_expressions:type_name -> spark.connect.Expression\n\t4,   // 143: spark.connect.GroupMap.initial_input:type_name -> spark.connect.Relation\n\t89,  // 144: spark.connect.GroupMap.initial_grouping_expressions:type_name -> spark.connect.Expression\n\t92,  // 145: spark.connect.GroupMap.state_schema:type_name -> spark.connect.DataType\n\t55,  // 146: spark.connect.GroupMap.transform_with_state_info:type_name -> spark.connect.TransformWithStateInfo\n\t92,  // 147: spark.connect.TransformWithStateInfo.output_schema:type_name -> spark.connect.DataType\n\t4,   // 148: spark.connect.CoGroupMap.input:type_name -> spark.connect.Relation\n\t89,  // 149: spark.connect.CoGroupMap.input_grouping_expressions:type_name -> spark.connect.Expression\n\t4,   // 150: spark.connect.CoGroupMap.other:type_name -> spark.connect.Relation\n\t89,  // 151: spark.connect.CoGroupMap.other_grouping_expressions:type_name -> spark.connect.Expression\n\t93,  // 152: spark.connect.CoGroupMap.func:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t89,  // 153: spark.connect.CoGroupMap.input_sorting_expressions:type_name -> spark.connect.Expression\n\t89,  // 154: spark.connect.CoGroupMap.other_sorting_expressions:type_name -> spark.connect.Expression\n\t4,   // 155: spark.connect.ApplyInPandasWithState.input:type_name -> spark.connect.Relation\n\t89,  // 156: spark.connect.ApplyInPandasWithState.grouping_expressions:type_name -> spark.connect.Expression\n\t93,  // 157: spark.connect.ApplyInPandasWithState.func:type_name -> spark.connect.CommonInlineUserDefinedFunction\n\t89,  // 158: spark.connect.CommonInlineUserDefinedTableFunction.arguments:type_name -> spark.connect.Expression\n\t59,  // 159: spark.connect.CommonInlineUserDefinedTableFunction.python_udtf:type_name -> spark.connect.PythonUDTF\n\t92,  // 160: spark.connect.PythonUDTF.return_type:type_name -> spark.connect.DataType\n\t61,  // 161: spark.connect.CommonInlineUserDefinedDataSource.python_data_source:type_name -> spark.connect.PythonDataSource\n\t4,   // 162: spark.connect.CollectMetrics.input:type_name -> spark.connect.Relation\n\t89,  // 163: spark.connect.CollectMetrics.metrics:type_name -> spark.connect.Expression\n\t4,   // 164: spark.connect.Parse.input:type_name -> spark.connect.Relation\n\t3,   // 165: spark.connect.Parse.format:type_name -> spark.connect.Parse.ParseFormat\n\t92,  // 166: spark.connect.Parse.schema:type_name -> spark.connect.DataType\n\t83,  // 167: spark.connect.Parse.options:type_name -> spark.connect.Parse.OptionsEntry\n\t4,   // 168: spark.connect.AsOfJoin.left:type_name -> spark.connect.Relation\n\t4,   // 169: spark.connect.AsOfJoin.right:type_name -> spark.connect.Relation\n\t89,  // 170: spark.connect.AsOfJoin.left_as_of:type_name -> spark.connect.Expression\n\t89,  // 171: spark.connect.AsOfJoin.right_as_of:type_name -> spark.connect.Expression\n\t89,  // 172: spark.connect.AsOfJoin.join_expr:type_name -> spark.connect.Expression\n\t89,  // 173: spark.connect.AsOfJoin.tolerance:type_name -> spark.connect.Expression\n\t4,   // 174: spark.connect.LateralJoin.left:type_name -> spark.connect.Relation\n\t4,   // 175: spark.connect.LateralJoin.right:type_name -> spark.connect.Relation\n\t89,  // 176: spark.connect.LateralJoin.join_condition:type_name -> spark.connect.Expression\n\t0,   // 177: spark.connect.LateralJoin.join_type:type_name -> spark.connect.Join.JoinType\n\t86,  // 178: spark.connect.MlRelation.Transform.obj_ref:type_name -> spark.connect.ObjectRef\n\t94,  // 179: spark.connect.MlRelation.Transform.transformer:type_name -> spark.connect.MlOperator\n\t4,   // 180: spark.connect.MlRelation.Transform.input:type_name -> spark.connect.Relation\n\t95,  // 181: spark.connect.MlRelation.Transform.params:type_name -> spark.connect.MlParams\n\t68,  // 182: spark.connect.Fetch.Method.args:type_name -> spark.connect.Fetch.Method.Args\n\t88,  // 183: spark.connect.Fetch.Method.Args.param:type_name -> spark.connect.Expression.Literal\n\t4,   // 184: spark.connect.Fetch.Method.Args.input:type_name -> spark.connect.Relation\n\t88,  // 185: spark.connect.SQL.ArgsEntry.value:type_name -> spark.connect.Expression.Literal\n\t89,  // 186: spark.connect.SQL.NamedArgumentsEntry.value:type_name -> spark.connect.Expression\n\t73,  // 187: spark.connect.Read.NamedTable.options:type_name -> spark.connect.Read.NamedTable.OptionsEntry\n\t74,  // 188: spark.connect.Read.DataSource.options:type_name -> spark.connect.Read.DataSource.OptionsEntry\n\t89,  // 189: spark.connect.Aggregate.Pivot.col:type_name -> spark.connect.Expression\n\t88,  // 190: spark.connect.Aggregate.Pivot.values:type_name -> spark.connect.Expression.Literal\n\t89,  // 191: spark.connect.Aggregate.GroupingSets.grouping_set:type_name -> spark.connect.Expression\n\t88,  // 192: spark.connect.StatSampleBy.Fraction.stratum:type_name -> spark.connect.Expression.Literal\n\t88,  // 193: spark.connect.NAReplace.Replacement.old_value:type_name -> spark.connect.Expression.Literal\n\t88,  // 194: spark.connect.NAReplace.Replacement.new_value:type_name -> spark.connect.Expression.Literal\n\t89,  // 195: spark.connect.Unpivot.Values.values:type_name -> spark.connect.Expression\n\t196, // [196:196] is the sub-list for method output_type\n\t196, // [196:196] is the sub-list for method input_type\n\t196, // [196:196] is the sub-list for extension type_name\n\t196, // [196:196] is the sub-list for extension extendee\n\t0,   // [0:196] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_relations_proto_init() }\nfunc file_spark_connect_relations_proto_init() {\n\tif File_spark_connect_relations_proto != nil {\n\t\treturn\n\t}\n\tfile_spark_connect_expressions_proto_init()\n\tfile_spark_connect_types_proto_init()\n\tfile_spark_connect_catalog_proto_init()\n\tfile_spark_connect_common_proto_init()\n\tfile_spark_connect_ml_common_proto_init()\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_relations_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Relation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlRelation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Fetch); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Unknown); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RelationCommon); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SQL); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WithRelations); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Read); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Project); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Filter); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Join); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SetOperation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Limit); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Offset); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Tail); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Aggregate); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Sort); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Drop); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Deduplicate); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*LocalRelation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CachedLocalRelation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CachedRemoteRelation); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Sample); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Range); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*SubqueryAlias); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Repartition); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ShowString); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*HtmlString); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatSummary); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatDescribe); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatCrosstab); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatCov); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatCorr); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatApproxQuantile); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatFreqItems); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatSampleBy); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*NAFill); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*NADrop); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*NAReplace); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ToDF); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WithColumnsRenamed); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WithColumns); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WithWatermark); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Hint); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Unpivot); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Transpose); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*UnresolvedTableValuedFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ToSchema); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*RepartitionByExpression); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MapPartitions); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*GroupMap); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*TransformWithStateInfo); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CoGroupMap); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*ApplyInPandasWithState); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CommonInlineUserDefinedTableFunction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PythonUDTF); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CommonInlineUserDefinedDataSource); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*PythonDataSource); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*CollectMetrics); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Parse); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*AsOfJoin); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*LateralJoin); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*MlRelation_Transform); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Fetch_Method); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Fetch_Method_Args); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Read_NamedTable); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Read_DataSource); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Join_JoinDataType); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Aggregate_Pivot); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Aggregate_GroupingSets); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*StatSampleBy_Fraction); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*NAReplace_Replacement); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*WithColumnsRenamed_Rename); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_relations_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*Unpivot_Values); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*Relation_Read)(nil),\n\t\t(*Relation_Project)(nil),\n\t\t(*Relation_Filter)(nil),\n\t\t(*Relation_Join)(nil),\n\t\t(*Relation_SetOp)(nil),\n\t\t(*Relation_Sort)(nil),\n\t\t(*Relation_Limit)(nil),\n\t\t(*Relation_Aggregate)(nil),\n\t\t(*Relation_Sql)(nil),\n\t\t(*Relation_LocalRelation)(nil),\n\t\t(*Relation_Sample)(nil),\n\t\t(*Relation_Offset)(nil),\n\t\t(*Relation_Deduplicate)(nil),\n\t\t(*Relation_Range)(nil),\n\t\t(*Relation_SubqueryAlias)(nil),\n\t\t(*Relation_Repartition)(nil),\n\t\t(*Relation_ToDf)(nil),\n\t\t(*Relation_WithColumnsRenamed)(nil),\n\t\t(*Relation_ShowString)(nil),\n\t\t(*Relation_Drop)(nil),\n\t\t(*Relation_Tail)(nil),\n\t\t(*Relation_WithColumns)(nil),\n\t\t(*Relation_Hint)(nil),\n\t\t(*Relation_Unpivot)(nil),\n\t\t(*Relation_ToSchema)(nil),\n\t\t(*Relation_RepartitionByExpression)(nil),\n\t\t(*Relation_MapPartitions)(nil),\n\t\t(*Relation_CollectMetrics)(nil),\n\t\t(*Relation_Parse)(nil),\n\t\t(*Relation_GroupMap)(nil),\n\t\t(*Relation_CoGroupMap)(nil),\n\t\t(*Relation_WithWatermark)(nil),\n\t\t(*Relation_ApplyInPandasWithState)(nil),\n\t\t(*Relation_HtmlString)(nil),\n\t\t(*Relation_CachedLocalRelation)(nil),\n\t\t(*Relation_CachedRemoteRelation)(nil),\n\t\t(*Relation_CommonInlineUserDefinedTableFunction)(nil),\n\t\t(*Relation_AsOfJoin)(nil),\n\t\t(*Relation_CommonInlineUserDefinedDataSource)(nil),\n\t\t(*Relation_WithRelations)(nil),\n\t\t(*Relation_Transpose)(nil),\n\t\t(*Relation_UnresolvedTableValuedFunction)(nil),\n\t\t(*Relation_LateralJoin)(nil),\n\t\t(*Relation_FillNa)(nil),\n\t\t(*Relation_DropNa)(nil),\n\t\t(*Relation_Replace)(nil),\n\t\t(*Relation_Summary)(nil),\n\t\t(*Relation_Crosstab)(nil),\n\t\t(*Relation_Describe)(nil),\n\t\t(*Relation_Cov)(nil),\n\t\t(*Relation_Corr)(nil),\n\t\t(*Relation_ApproxQuantile)(nil),\n\t\t(*Relation_FreqItems)(nil),\n\t\t(*Relation_SampleBy)(nil),\n\t\t(*Relation_Catalog)(nil),\n\t\t(*Relation_MlRelation)(nil),\n\t\t(*Relation_Extension)(nil),\n\t\t(*Relation_Unknown)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[1].OneofWrappers = []interface{}{\n\t\t(*MlRelation_Transform_)(nil),\n\t\t(*MlRelation_Fetch)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[4].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[7].OneofWrappers = []interface{}{\n\t\t(*Read_NamedTable_)(nil),\n\t\t(*Read_DataSource_)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[10].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[11].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[16].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[18].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[19].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[22].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[23].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[25].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[32].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[34].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[35].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[37].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[44].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[48].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[49].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[50].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[51].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[54].OneofWrappers = []interface{}{\n\t\t(*CommonInlineUserDefinedTableFunction_PythonUdtf)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[55].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[56].OneofWrappers = []interface{}{\n\t\t(*CommonInlineUserDefinedDataSource_PythonDataSource)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[59].OneofWrappers = []interface{}{}\n\tfile_spark_connect_relations_proto_msgTypes[62].OneofWrappers = []interface{}{\n\t\t(*MlRelation_Transform_ObjRef)(nil),\n\t\t(*MlRelation_Transform_Transformer)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[64].OneofWrappers = []interface{}{\n\t\t(*Fetch_Method_Args_Param)(nil),\n\t\t(*Fetch_Method_Args_Input)(nil),\n\t}\n\tfile_spark_connect_relations_proto_msgTypes[68].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_relations_proto_rawDesc,\n\t\t\tNumEnums:      4,\n\t\t\tNumMessages:   80,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_relations_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_relations_proto_depIdxs,\n\t\tEnumInfos:         file_spark_connect_relations_proto_enumTypes,\n\t\tMessageInfos:      file_spark_connect_relations_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_relations_proto = out.File\n\tfile_spark_connect_relations_proto_rawDesc = nil\n\tfile_spark_connect_relations_proto_goTypes = nil\n\tfile_spark_connect_relations_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/generated/types.pb.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\n// Code generated by protoc-gen-go. DO NOT EDIT.\n// versions:\n// \tprotoc-gen-go v1.30.0\n// \tprotoc        (unknown)\n// source: spark/connect/types.proto\n\npackage generated\n\nimport (\n\tprotoreflect \"google.golang.org/protobuf/reflect/protoreflect\"\n\tprotoimpl \"google.golang.org/protobuf/runtime/protoimpl\"\n\treflect \"reflect\"\n\tsync \"sync\"\n)\n\nconst (\n\t// Verify that this generated code is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)\n\t// Verify that runtime/protoimpl is sufficiently up-to-date.\n\t_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)\n)\n\n// This message describes the logical [[DataType]] of something. It does not carry the value\n// itself but only describes it.\ntype DataType struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// Types that are assignable to Kind:\n\t//\n\t//\t*DataType_Null\n\t//\t*DataType_Binary_\n\t//\t*DataType_Boolean_\n\t//\t*DataType_Byte_\n\t//\t*DataType_Short_\n\t//\t*DataType_Integer_\n\t//\t*DataType_Long_\n\t//\t*DataType_Float_\n\t//\t*DataType_Double_\n\t//\t*DataType_Decimal_\n\t//\t*DataType_String_\n\t//\t*DataType_Char_\n\t//\t*DataType_VarChar_\n\t//\t*DataType_Date_\n\t//\t*DataType_Timestamp_\n\t//\t*DataType_TimestampNtz\n\t//\t*DataType_CalendarInterval_\n\t//\t*DataType_YearMonthInterval_\n\t//\t*DataType_DayTimeInterval_\n\t//\t*DataType_Array_\n\t//\t*DataType_Struct_\n\t//\t*DataType_Map_\n\t//\t*DataType_Variant_\n\t//\t*DataType_Udt\n\t//\t*DataType_Unparsed_\n\tKind isDataType_Kind `protobuf_oneof:\"kind\"`\n}\n\nfunc (x *DataType) Reset() {\n\t*x = DataType{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[0]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType) ProtoMessage() {}\n\nfunc (x *DataType) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[0]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType.ProtoReflect.Descriptor instead.\nfunc (*DataType) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0}\n}\n\nfunc (m *DataType) GetKind() isDataType_Kind {\n\tif m != nil {\n\t\treturn m.Kind\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetNull() *DataType_NULL {\n\tif x, ok := x.GetKind().(*DataType_Null); ok {\n\t\treturn x.Null\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetBinary() *DataType_Binary {\n\tif x, ok := x.GetKind().(*DataType_Binary_); ok {\n\t\treturn x.Binary\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetBoolean() *DataType_Boolean {\n\tif x, ok := x.GetKind().(*DataType_Boolean_); ok {\n\t\treturn x.Boolean\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetByte() *DataType_Byte {\n\tif x, ok := x.GetKind().(*DataType_Byte_); ok {\n\t\treturn x.Byte\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetShort() *DataType_Short {\n\tif x, ok := x.GetKind().(*DataType_Short_); ok {\n\t\treturn x.Short\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetInteger() *DataType_Integer {\n\tif x, ok := x.GetKind().(*DataType_Integer_); ok {\n\t\treturn x.Integer\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetLong() *DataType_Long {\n\tif x, ok := x.GetKind().(*DataType_Long_); ok {\n\t\treturn x.Long\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetFloat() *DataType_Float {\n\tif x, ok := x.GetKind().(*DataType_Float_); ok {\n\t\treturn x.Float\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetDouble() *DataType_Double {\n\tif x, ok := x.GetKind().(*DataType_Double_); ok {\n\t\treturn x.Double\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetDecimal() *DataType_Decimal {\n\tif x, ok := x.GetKind().(*DataType_Decimal_); ok {\n\t\treturn x.Decimal\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetString_() *DataType_String {\n\tif x, ok := x.GetKind().(*DataType_String_); ok {\n\t\treturn x.String_\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetChar() *DataType_Char {\n\tif x, ok := x.GetKind().(*DataType_Char_); ok {\n\t\treturn x.Char\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetVarChar() *DataType_VarChar {\n\tif x, ok := x.GetKind().(*DataType_VarChar_); ok {\n\t\treturn x.VarChar\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetDate() *DataType_Date {\n\tif x, ok := x.GetKind().(*DataType_Date_); ok {\n\t\treturn x.Date\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetTimestamp() *DataType_Timestamp {\n\tif x, ok := x.GetKind().(*DataType_Timestamp_); ok {\n\t\treturn x.Timestamp\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetTimestampNtz() *DataType_TimestampNTZ {\n\tif x, ok := x.GetKind().(*DataType_TimestampNtz); ok {\n\t\treturn x.TimestampNtz\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetCalendarInterval() *DataType_CalendarInterval {\n\tif x, ok := x.GetKind().(*DataType_CalendarInterval_); ok {\n\t\treturn x.CalendarInterval\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetYearMonthInterval() *DataType_YearMonthInterval {\n\tif x, ok := x.GetKind().(*DataType_YearMonthInterval_); ok {\n\t\treturn x.YearMonthInterval\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetDayTimeInterval() *DataType_DayTimeInterval {\n\tif x, ok := x.GetKind().(*DataType_DayTimeInterval_); ok {\n\t\treturn x.DayTimeInterval\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetArray() *DataType_Array {\n\tif x, ok := x.GetKind().(*DataType_Array_); ok {\n\t\treturn x.Array\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetStruct() *DataType_Struct {\n\tif x, ok := x.GetKind().(*DataType_Struct_); ok {\n\t\treturn x.Struct\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetMap() *DataType_Map {\n\tif x, ok := x.GetKind().(*DataType_Map_); ok {\n\t\treturn x.Map\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetVariant() *DataType_Variant {\n\tif x, ok := x.GetKind().(*DataType_Variant_); ok {\n\t\treturn x.Variant\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetUdt() *DataType_UDT {\n\tif x, ok := x.GetKind().(*DataType_Udt); ok {\n\t\treturn x.Udt\n\t}\n\treturn nil\n}\n\nfunc (x *DataType) GetUnparsed() *DataType_Unparsed {\n\tif x, ok := x.GetKind().(*DataType_Unparsed_); ok {\n\t\treturn x.Unparsed\n\t}\n\treturn nil\n}\n\ntype isDataType_Kind interface {\n\tisDataType_Kind()\n}\n\ntype DataType_Null struct {\n\tNull *DataType_NULL `protobuf:\"bytes,1,opt,name=null,proto3,oneof\"`\n}\n\ntype DataType_Binary_ struct {\n\tBinary *DataType_Binary `protobuf:\"bytes,2,opt,name=binary,proto3,oneof\"`\n}\n\ntype DataType_Boolean_ struct {\n\tBoolean *DataType_Boolean `protobuf:\"bytes,3,opt,name=boolean,proto3,oneof\"`\n}\n\ntype DataType_Byte_ struct {\n\t// Numeric types\n\tByte *DataType_Byte `protobuf:\"bytes,4,opt,name=byte,proto3,oneof\"`\n}\n\ntype DataType_Short_ struct {\n\tShort *DataType_Short `protobuf:\"bytes,5,opt,name=short,proto3,oneof\"`\n}\n\ntype DataType_Integer_ struct {\n\tInteger *DataType_Integer `protobuf:\"bytes,6,opt,name=integer,proto3,oneof\"`\n}\n\ntype DataType_Long_ struct {\n\tLong *DataType_Long `protobuf:\"bytes,7,opt,name=long,proto3,oneof\"`\n}\n\ntype DataType_Float_ struct {\n\tFloat *DataType_Float `protobuf:\"bytes,8,opt,name=float,proto3,oneof\"`\n}\n\ntype DataType_Double_ struct {\n\tDouble *DataType_Double `protobuf:\"bytes,9,opt,name=double,proto3,oneof\"`\n}\n\ntype DataType_Decimal_ struct {\n\tDecimal *DataType_Decimal `protobuf:\"bytes,10,opt,name=decimal,proto3,oneof\"`\n}\n\ntype DataType_String_ struct {\n\t// String types\n\tString_ *DataType_String `protobuf:\"bytes,11,opt,name=string,proto3,oneof\"`\n}\n\ntype DataType_Char_ struct {\n\tChar *DataType_Char `protobuf:\"bytes,12,opt,name=char,proto3,oneof\"`\n}\n\ntype DataType_VarChar_ struct {\n\tVarChar *DataType_VarChar `protobuf:\"bytes,13,opt,name=var_char,json=varChar,proto3,oneof\"`\n}\n\ntype DataType_Date_ struct {\n\t// Datatime types\n\tDate *DataType_Date `protobuf:\"bytes,14,opt,name=date,proto3,oneof\"`\n}\n\ntype DataType_Timestamp_ struct {\n\tTimestamp *DataType_Timestamp `protobuf:\"bytes,15,opt,name=timestamp,proto3,oneof\"`\n}\n\ntype DataType_TimestampNtz struct {\n\tTimestampNtz *DataType_TimestampNTZ `protobuf:\"bytes,16,opt,name=timestamp_ntz,json=timestampNtz,proto3,oneof\"`\n}\n\ntype DataType_CalendarInterval_ struct {\n\t// Interval types\n\tCalendarInterval *DataType_CalendarInterval `protobuf:\"bytes,17,opt,name=calendar_interval,json=calendarInterval,proto3,oneof\"`\n}\n\ntype DataType_YearMonthInterval_ struct {\n\tYearMonthInterval *DataType_YearMonthInterval `protobuf:\"bytes,18,opt,name=year_month_interval,json=yearMonthInterval,proto3,oneof\"`\n}\n\ntype DataType_DayTimeInterval_ struct {\n\tDayTimeInterval *DataType_DayTimeInterval `protobuf:\"bytes,19,opt,name=day_time_interval,json=dayTimeInterval,proto3,oneof\"`\n}\n\ntype DataType_Array_ struct {\n\t// Complex types\n\tArray *DataType_Array `protobuf:\"bytes,20,opt,name=array,proto3,oneof\"`\n}\n\ntype DataType_Struct_ struct {\n\tStruct *DataType_Struct `protobuf:\"bytes,21,opt,name=struct,proto3,oneof\"`\n}\n\ntype DataType_Map_ struct {\n\tMap *DataType_Map `protobuf:\"bytes,22,opt,name=map,proto3,oneof\"`\n}\n\ntype DataType_Variant_ struct {\n\tVariant *DataType_Variant `protobuf:\"bytes,25,opt,name=variant,proto3,oneof\"`\n}\n\ntype DataType_Udt struct {\n\t// UserDefinedType\n\tUdt *DataType_UDT `protobuf:\"bytes,23,opt,name=udt,proto3,oneof\"`\n}\n\ntype DataType_Unparsed_ struct {\n\t// UnparsedDataType\n\tUnparsed *DataType_Unparsed `protobuf:\"bytes,24,opt,name=unparsed,proto3,oneof\"`\n}\n\nfunc (*DataType_Null) isDataType_Kind() {}\n\nfunc (*DataType_Binary_) isDataType_Kind() {}\n\nfunc (*DataType_Boolean_) isDataType_Kind() {}\n\nfunc (*DataType_Byte_) isDataType_Kind() {}\n\nfunc (*DataType_Short_) isDataType_Kind() {}\n\nfunc (*DataType_Integer_) isDataType_Kind() {}\n\nfunc (*DataType_Long_) isDataType_Kind() {}\n\nfunc (*DataType_Float_) isDataType_Kind() {}\n\nfunc (*DataType_Double_) isDataType_Kind() {}\n\nfunc (*DataType_Decimal_) isDataType_Kind() {}\n\nfunc (*DataType_String_) isDataType_Kind() {}\n\nfunc (*DataType_Char_) isDataType_Kind() {}\n\nfunc (*DataType_VarChar_) isDataType_Kind() {}\n\nfunc (*DataType_Date_) isDataType_Kind() {}\n\nfunc (*DataType_Timestamp_) isDataType_Kind() {}\n\nfunc (*DataType_TimestampNtz) isDataType_Kind() {}\n\nfunc (*DataType_CalendarInterval_) isDataType_Kind() {}\n\nfunc (*DataType_YearMonthInterval_) isDataType_Kind() {}\n\nfunc (*DataType_DayTimeInterval_) isDataType_Kind() {}\n\nfunc (*DataType_Array_) isDataType_Kind() {}\n\nfunc (*DataType_Struct_) isDataType_Kind() {}\n\nfunc (*DataType_Map_) isDataType_Kind() {}\n\nfunc (*DataType_Variant_) isDataType_Kind() {}\n\nfunc (*DataType_Udt) isDataType_Kind() {}\n\nfunc (*DataType_Unparsed_) isDataType_Kind() {}\n\ntype DataType_Boolean struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Boolean) Reset() {\n\t*x = DataType_Boolean{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[1]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Boolean) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Boolean) ProtoMessage() {}\n\nfunc (x *DataType_Boolean) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[1]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Boolean.ProtoReflect.Descriptor instead.\nfunc (*DataType_Boolean) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 0}\n}\n\nfunc (x *DataType_Boolean) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Byte struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Byte) Reset() {\n\t*x = DataType_Byte{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[2]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Byte) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Byte) ProtoMessage() {}\n\nfunc (x *DataType_Byte) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[2]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Byte.ProtoReflect.Descriptor instead.\nfunc (*DataType_Byte) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 1}\n}\n\nfunc (x *DataType_Byte) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Short struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Short) Reset() {\n\t*x = DataType_Short{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[3]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Short) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Short) ProtoMessage() {}\n\nfunc (x *DataType_Short) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[3]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Short.ProtoReflect.Descriptor instead.\nfunc (*DataType_Short) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 2}\n}\n\nfunc (x *DataType_Short) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Integer struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Integer) Reset() {\n\t*x = DataType_Integer{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[4]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Integer) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Integer) ProtoMessage() {}\n\nfunc (x *DataType_Integer) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[4]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Integer.ProtoReflect.Descriptor instead.\nfunc (*DataType_Integer) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 3}\n}\n\nfunc (x *DataType_Integer) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Long struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Long) Reset() {\n\t*x = DataType_Long{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[5]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Long) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Long) ProtoMessage() {}\n\nfunc (x *DataType_Long) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[5]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Long.ProtoReflect.Descriptor instead.\nfunc (*DataType_Long) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 4}\n}\n\nfunc (x *DataType_Long) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Float struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Float) Reset() {\n\t*x = DataType_Float{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[6]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Float) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Float) ProtoMessage() {}\n\nfunc (x *DataType_Float) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[6]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Float.ProtoReflect.Descriptor instead.\nfunc (*DataType_Float) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 5}\n}\n\nfunc (x *DataType_Float) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Double struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Double) Reset() {\n\t*x = DataType_Double{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[7]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Double) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Double) ProtoMessage() {}\n\nfunc (x *DataType_Double) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[7]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Double.ProtoReflect.Descriptor instead.\nfunc (*DataType_Double) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 6}\n}\n\nfunc (x *DataType_Double) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_String struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n\tCollation              string `protobuf:\"bytes,2,opt,name=collation,proto3\" json:\"collation,omitempty\"`\n}\n\nfunc (x *DataType_String) Reset() {\n\t*x = DataType_String{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[8]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_String) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_String) ProtoMessage() {}\n\nfunc (x *DataType_String) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[8]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_String.ProtoReflect.Descriptor instead.\nfunc (*DataType_String) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 7}\n}\n\nfunc (x *DataType_String) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_String) GetCollation() string {\n\tif x != nil {\n\t\treturn x.Collation\n\t}\n\treturn \"\"\n}\n\ntype DataType_Binary struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Binary) Reset() {\n\t*x = DataType_Binary{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[9]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Binary) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Binary) ProtoMessage() {}\n\nfunc (x *DataType_Binary) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[9]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Binary.ProtoReflect.Descriptor instead.\nfunc (*DataType_Binary) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 8}\n}\n\nfunc (x *DataType_Binary) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_NULL struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_NULL) Reset() {\n\t*x = DataType_NULL{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[10]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_NULL) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_NULL) ProtoMessage() {}\n\nfunc (x *DataType_NULL) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[10]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_NULL.ProtoReflect.Descriptor instead.\nfunc (*DataType_NULL) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 9}\n}\n\nfunc (x *DataType_NULL) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Timestamp struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Timestamp) Reset() {\n\t*x = DataType_Timestamp{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[11]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Timestamp) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Timestamp) ProtoMessage() {}\n\nfunc (x *DataType_Timestamp) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[11]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Timestamp.ProtoReflect.Descriptor instead.\nfunc (*DataType_Timestamp) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 10}\n}\n\nfunc (x *DataType_Timestamp) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Date struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Date) Reset() {\n\t*x = DataType_Date{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[12]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Date) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Date) ProtoMessage() {}\n\nfunc (x *DataType_Date) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[12]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Date.ProtoReflect.Descriptor instead.\nfunc (*DataType_Date) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 11}\n}\n\nfunc (x *DataType_Date) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_TimestampNTZ struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_TimestampNTZ) Reset() {\n\t*x = DataType_TimestampNTZ{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[13]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_TimestampNTZ) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_TimestampNTZ) ProtoMessage() {}\n\nfunc (x *DataType_TimestampNTZ) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[13]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_TimestampNTZ.ProtoReflect.Descriptor instead.\nfunc (*DataType_TimestampNTZ) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 12}\n}\n\nfunc (x *DataType_TimestampNTZ) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_CalendarInterval struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_CalendarInterval) Reset() {\n\t*x = DataType_CalendarInterval{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[14]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_CalendarInterval) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_CalendarInterval) ProtoMessage() {}\n\nfunc (x *DataType_CalendarInterval) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[14]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_CalendarInterval.ProtoReflect.Descriptor instead.\nfunc (*DataType_CalendarInterval) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 13}\n}\n\nfunc (x *DataType_CalendarInterval) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_YearMonthInterval struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tStartField             *int32 `protobuf:\"varint,1,opt,name=start_field,json=startField,proto3,oneof\" json:\"start_field,omitempty\"`\n\tEndField               *int32 `protobuf:\"varint,2,opt,name=end_field,json=endField,proto3,oneof\" json:\"end_field,omitempty\"`\n\tTypeVariationReference uint32 `protobuf:\"varint,3,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_YearMonthInterval) Reset() {\n\t*x = DataType_YearMonthInterval{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[15]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_YearMonthInterval) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_YearMonthInterval) ProtoMessage() {}\n\nfunc (x *DataType_YearMonthInterval) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[15]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_YearMonthInterval.ProtoReflect.Descriptor instead.\nfunc (*DataType_YearMonthInterval) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 14}\n}\n\nfunc (x *DataType_YearMonthInterval) GetStartField() int32 {\n\tif x != nil && x.StartField != nil {\n\t\treturn *x.StartField\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_YearMonthInterval) GetEndField() int32 {\n\tif x != nil && x.EndField != nil {\n\t\treturn *x.EndField\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_YearMonthInterval) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_DayTimeInterval struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tStartField             *int32 `protobuf:\"varint,1,opt,name=start_field,json=startField,proto3,oneof\" json:\"start_field,omitempty\"`\n\tEndField               *int32 `protobuf:\"varint,2,opt,name=end_field,json=endField,proto3,oneof\" json:\"end_field,omitempty\"`\n\tTypeVariationReference uint32 `protobuf:\"varint,3,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_DayTimeInterval) Reset() {\n\t*x = DataType_DayTimeInterval{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[16]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_DayTimeInterval) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_DayTimeInterval) ProtoMessage() {}\n\nfunc (x *DataType_DayTimeInterval) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[16]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_DayTimeInterval.ProtoReflect.Descriptor instead.\nfunc (*DataType_DayTimeInterval) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 15}\n}\n\nfunc (x *DataType_DayTimeInterval) GetStartField() int32 {\n\tif x != nil && x.StartField != nil {\n\t\treturn *x.StartField\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_DayTimeInterval) GetEndField() int32 {\n\tif x != nil && x.EndField != nil {\n\t\treturn *x.EndField\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_DayTimeInterval) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\n// Start compound types.\ntype DataType_Char struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tLength                 int32  `protobuf:\"varint,1,opt,name=length,proto3\" json:\"length,omitempty\"`\n\tTypeVariationReference uint32 `protobuf:\"varint,2,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Char) Reset() {\n\t*x = DataType_Char{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[17]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Char) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Char) ProtoMessage() {}\n\nfunc (x *DataType_Char) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[17]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Char.ProtoReflect.Descriptor instead.\nfunc (*DataType_Char) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 16}\n}\n\nfunc (x *DataType_Char) GetLength() int32 {\n\tif x != nil {\n\t\treturn x.Length\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_Char) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_VarChar struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tLength                 int32  `protobuf:\"varint,1,opt,name=length,proto3\" json:\"length,omitempty\"`\n\tTypeVariationReference uint32 `protobuf:\"varint,2,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_VarChar) Reset() {\n\t*x = DataType_VarChar{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[18]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_VarChar) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_VarChar) ProtoMessage() {}\n\nfunc (x *DataType_VarChar) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[18]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_VarChar.ProtoReflect.Descriptor instead.\nfunc (*DataType_VarChar) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 17}\n}\n\nfunc (x *DataType_VarChar) GetLength() int32 {\n\tif x != nil {\n\t\treturn x.Length\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_VarChar) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Decimal struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tScale                  *int32 `protobuf:\"varint,1,opt,name=scale,proto3,oneof\" json:\"scale,omitempty\"`\n\tPrecision              *int32 `protobuf:\"varint,2,opt,name=precision,proto3,oneof\" json:\"precision,omitempty\"`\n\tTypeVariationReference uint32 `protobuf:\"varint,3,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Decimal) Reset() {\n\t*x = DataType_Decimal{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[19]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Decimal) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Decimal) ProtoMessage() {}\n\nfunc (x *DataType_Decimal) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[19]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Decimal.ProtoReflect.Descriptor instead.\nfunc (*DataType_Decimal) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 18}\n}\n\nfunc (x *DataType_Decimal) GetScale() int32 {\n\tif x != nil && x.Scale != nil {\n\t\treturn *x.Scale\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_Decimal) GetPrecision() int32 {\n\tif x != nil && x.Precision != nil {\n\t\treturn *x.Precision\n\t}\n\treturn 0\n}\n\nfunc (x *DataType_Decimal) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_StructField struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tName     string    `protobuf:\"bytes,1,opt,name=name,proto3\" json:\"name,omitempty\"`\n\tDataType *DataType `protobuf:\"bytes,2,opt,name=data_type,json=dataType,proto3\" json:\"data_type,omitempty\"`\n\tNullable bool      `protobuf:\"varint,3,opt,name=nullable,proto3\" json:\"nullable,omitempty\"`\n\tMetadata *string   `protobuf:\"bytes,4,opt,name=metadata,proto3,oneof\" json:\"metadata,omitempty\"`\n}\n\nfunc (x *DataType_StructField) Reset() {\n\t*x = DataType_StructField{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[20]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_StructField) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_StructField) ProtoMessage() {}\n\nfunc (x *DataType_StructField) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[20]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_StructField.ProtoReflect.Descriptor instead.\nfunc (*DataType_StructField) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 19}\n}\n\nfunc (x *DataType_StructField) GetName() string {\n\tif x != nil {\n\t\treturn x.Name\n\t}\n\treturn \"\"\n}\n\nfunc (x *DataType_StructField) GetDataType() *DataType {\n\tif x != nil {\n\t\treturn x.DataType\n\t}\n\treturn nil\n}\n\nfunc (x *DataType_StructField) GetNullable() bool {\n\tif x != nil {\n\t\treturn x.Nullable\n\t}\n\treturn false\n}\n\nfunc (x *DataType_StructField) GetMetadata() string {\n\tif x != nil && x.Metadata != nil {\n\t\treturn *x.Metadata\n\t}\n\treturn \"\"\n}\n\ntype DataType_Struct struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tFields                 []*DataType_StructField `protobuf:\"bytes,1,rep,name=fields,proto3\" json:\"fields,omitempty\"`\n\tTypeVariationReference uint32                  `protobuf:\"varint,2,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Struct) Reset() {\n\t*x = DataType_Struct{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[21]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Struct) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Struct) ProtoMessage() {}\n\nfunc (x *DataType_Struct) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[21]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Struct.ProtoReflect.Descriptor instead.\nfunc (*DataType_Struct) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 20}\n}\n\nfunc (x *DataType_Struct) GetFields() []*DataType_StructField {\n\tif x != nil {\n\t\treturn x.Fields\n\t}\n\treturn nil\n}\n\nfunc (x *DataType_Struct) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Array struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tElementType            *DataType `protobuf:\"bytes,1,opt,name=element_type,json=elementType,proto3\" json:\"element_type,omitempty\"`\n\tContainsNull           bool      `protobuf:\"varint,2,opt,name=contains_null,json=containsNull,proto3\" json:\"contains_null,omitempty\"`\n\tTypeVariationReference uint32    `protobuf:\"varint,3,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Array) Reset() {\n\t*x = DataType_Array{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[22]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Array) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Array) ProtoMessage() {}\n\nfunc (x *DataType_Array) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[22]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Array.ProtoReflect.Descriptor instead.\nfunc (*DataType_Array) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 21}\n}\n\nfunc (x *DataType_Array) GetElementType() *DataType {\n\tif x != nil {\n\t\treturn x.ElementType\n\t}\n\treturn nil\n}\n\nfunc (x *DataType_Array) GetContainsNull() bool {\n\tif x != nil {\n\t\treturn x.ContainsNull\n\t}\n\treturn false\n}\n\nfunc (x *DataType_Array) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Map struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tKeyType                *DataType `protobuf:\"bytes,1,opt,name=key_type,json=keyType,proto3\" json:\"key_type,omitempty\"`\n\tValueType              *DataType `protobuf:\"bytes,2,opt,name=value_type,json=valueType,proto3\" json:\"value_type,omitempty\"`\n\tValueContainsNull      bool      `protobuf:\"varint,3,opt,name=value_contains_null,json=valueContainsNull,proto3\" json:\"value_contains_null,omitempty\"`\n\tTypeVariationReference uint32    `protobuf:\"varint,4,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Map) Reset() {\n\t*x = DataType_Map{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[23]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Map) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Map) ProtoMessage() {}\n\nfunc (x *DataType_Map) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[23]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Map.ProtoReflect.Descriptor instead.\nfunc (*DataType_Map) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 22}\n}\n\nfunc (x *DataType_Map) GetKeyType() *DataType {\n\tif x != nil {\n\t\treturn x.KeyType\n\t}\n\treturn nil\n}\n\nfunc (x *DataType_Map) GetValueType() *DataType {\n\tif x != nil {\n\t\treturn x.ValueType\n\t}\n\treturn nil\n}\n\nfunc (x *DataType_Map) GetValueContainsNull() bool {\n\tif x != nil {\n\t\treturn x.ValueContainsNull\n\t}\n\treturn false\n}\n\nfunc (x *DataType_Map) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_Variant struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tTypeVariationReference uint32 `protobuf:\"varint,1,opt,name=type_variation_reference,json=typeVariationReference,proto3\" json:\"type_variation_reference,omitempty\"`\n}\n\nfunc (x *DataType_Variant) Reset() {\n\t*x = DataType_Variant{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[24]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Variant) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Variant) ProtoMessage() {}\n\nfunc (x *DataType_Variant) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[24]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Variant.ProtoReflect.Descriptor instead.\nfunc (*DataType_Variant) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 23}\n}\n\nfunc (x *DataType_Variant) GetTypeVariationReference() uint32 {\n\tif x != nil {\n\t\treturn x.TypeVariationReference\n\t}\n\treturn 0\n}\n\ntype DataType_UDT struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\tType string `protobuf:\"bytes,1,opt,name=type,proto3\" json:\"type,omitempty\"`\n\t// Required for Scala/Java UDT\n\tJvmClass *string `protobuf:\"bytes,2,opt,name=jvm_class,json=jvmClass,proto3,oneof\" json:\"jvm_class,omitempty\"`\n\t// Required for Python UDT\n\tPythonClass *string `protobuf:\"bytes,3,opt,name=python_class,json=pythonClass,proto3,oneof\" json:\"python_class,omitempty\"`\n\t// Required for Python UDT\n\tSerializedPythonClass *string `protobuf:\"bytes,4,opt,name=serialized_python_class,json=serializedPythonClass,proto3,oneof\" json:\"serialized_python_class,omitempty\"`\n\t// Required for Python UDT\n\tSqlType *DataType `protobuf:\"bytes,5,opt,name=sql_type,json=sqlType,proto3,oneof\" json:\"sql_type,omitempty\"`\n}\n\nfunc (x *DataType_UDT) Reset() {\n\t*x = DataType_UDT{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[25]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_UDT) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_UDT) ProtoMessage() {}\n\nfunc (x *DataType_UDT) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[25]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_UDT.ProtoReflect.Descriptor instead.\nfunc (*DataType_UDT) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 24}\n}\n\nfunc (x *DataType_UDT) GetType() string {\n\tif x != nil {\n\t\treturn x.Type\n\t}\n\treturn \"\"\n}\n\nfunc (x *DataType_UDT) GetJvmClass() string {\n\tif x != nil && x.JvmClass != nil {\n\t\treturn *x.JvmClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *DataType_UDT) GetPythonClass() string {\n\tif x != nil && x.PythonClass != nil {\n\t\treturn *x.PythonClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *DataType_UDT) GetSerializedPythonClass() string {\n\tif x != nil && x.SerializedPythonClass != nil {\n\t\treturn *x.SerializedPythonClass\n\t}\n\treturn \"\"\n}\n\nfunc (x *DataType_UDT) GetSqlType() *DataType {\n\tif x != nil {\n\t\treturn x.SqlType\n\t}\n\treturn nil\n}\n\ntype DataType_Unparsed struct {\n\tstate         protoimpl.MessageState\n\tsizeCache     protoimpl.SizeCache\n\tunknownFields protoimpl.UnknownFields\n\n\t// (Required) The unparsed data type string\n\tDataTypeString string `protobuf:\"bytes,1,opt,name=data_type_string,json=dataTypeString,proto3\" json:\"data_type_string,omitempty\"`\n}\n\nfunc (x *DataType_Unparsed) Reset() {\n\t*x = DataType_Unparsed{}\n\tif protoimpl.UnsafeEnabled {\n\t\tmi := &file_spark_connect_types_proto_msgTypes[26]\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tms.StoreMessageInfo(mi)\n\t}\n}\n\nfunc (x *DataType_Unparsed) String() string {\n\treturn protoimpl.X.MessageStringOf(x)\n}\n\nfunc (*DataType_Unparsed) ProtoMessage() {}\n\nfunc (x *DataType_Unparsed) ProtoReflect() protoreflect.Message {\n\tmi := &file_spark_connect_types_proto_msgTypes[26]\n\tif protoimpl.UnsafeEnabled && x != nil {\n\t\tms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))\n\t\tif ms.LoadMessageInfo() == nil {\n\t\t\tms.StoreMessageInfo(mi)\n\t\t}\n\t\treturn ms\n\t}\n\treturn mi.MessageOf(x)\n}\n\n// Deprecated: Use DataType_Unparsed.ProtoReflect.Descriptor instead.\nfunc (*DataType_Unparsed) Descriptor() ([]byte, []int) {\n\treturn file_spark_connect_types_proto_rawDescGZIP(), []int{0, 25}\n}\n\nfunc (x *DataType_Unparsed) GetDataTypeString() string {\n\tif x != nil {\n\t\treturn x.DataTypeString\n\t}\n\treturn \"\"\n}\n\nvar File_spark_connect_types_proto protoreflect.FileDescriptor\n\nvar file_spark_connect_types_proto_rawDesc = []byte{\n\t0x0a, 0x19, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,\n\t0x74, 0x79, 0x70, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0d, 0x73, 0x70, 0x61,\n\t0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x22, 0xf9, 0x21, 0x0a, 0x08, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x12, 0x32, 0x0a, 0x04, 0x6e, 0x75, 0x6c, 0x6c, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x4e,\n\t0x55, 0x4c, 0x4c, 0x48, 0x00, 0x52, 0x04, 0x6e, 0x75, 0x6c, 0x6c, 0x12, 0x38, 0x0a, 0x06, 0x62,\n\t0x69, 0x6e, 0x61, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61,\n\t0x54, 0x79, 0x70, 0x65, 0x2e, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x48, 0x00, 0x52, 0x06, 0x62,\n\t0x69, 0x6e, 0x61, 0x72, 0x79, 0x12, 0x3b, 0x0a, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x65, 0x61, 0x6e,\n\t0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e,\n\t0x42, 0x6f, 0x6f, 0x6c, 0x65, 0x61, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x65,\n\t0x61, 0x6e, 0x12, 0x32, 0x0a, 0x04, 0x62, 0x79, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x42, 0x79, 0x74, 0x65, 0x48, 0x00,\n\t0x52, 0x04, 0x62, 0x79, 0x74, 0x65, 0x12, 0x35, 0x0a, 0x05, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x18,\n\t0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x53,\n\t0x68, 0x6f, 0x72, 0x74, 0x48, 0x00, 0x52, 0x05, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x12, 0x3b, 0x0a,\n\t0x07, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x48,\n\t0x00, 0x52, 0x07, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x12, 0x32, 0x0a, 0x04, 0x6c, 0x6f,\n\t0x6e, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70,\n\t0x65, 0x2e, 0x4c, 0x6f, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x04, 0x6c, 0x6f, 0x6e, 0x67, 0x12, 0x35,\n\t0x0a, 0x05, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61,\n\t0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x48, 0x00, 0x52, 0x05,\n\t0x66, 0x6c, 0x6f, 0x61, 0x74, 0x12, 0x38, 0x0a, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x18,\n\t0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x44,\n\t0x6f, 0x75, 0x62, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x12,\n\t0x3b, 0x0a, 0x07, 0x64, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1f, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x44, 0x65, 0x63, 0x69, 0x6d, 0x61,\n\t0x6c, 0x48, 0x00, 0x52, 0x07, 0x64, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x12, 0x38, 0x0a, 0x06,\n\t0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74,\n\t0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x06,\n\t0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x32, 0x0a, 0x04, 0x63, 0x68, 0x61, 0x72, 0x18, 0x0c,\n\t0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e,\n\t0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x43, 0x68,\n\t0x61, 0x72, 0x48, 0x00, 0x52, 0x04, 0x63, 0x68, 0x61, 0x72, 0x12, 0x3c, 0x0a, 0x08, 0x76, 0x61,\n\t0x72, 0x5f, 0x63, 0x68, 0x61, 0x72, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74,\n\t0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x56, 0x61, 0x72, 0x43, 0x68, 0x61, 0x72, 0x48, 0x00, 0x52,\n\t0x07, 0x76, 0x61, 0x72, 0x43, 0x68, 0x61, 0x72, 0x12, 0x32, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x65,\n\t0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e,\n\t0x44, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x65, 0x12, 0x41, 0x0a, 0x09,\n\t0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32,\n\t0x21, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,\n\t0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,\n\t0x6d, 0x70, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12,\n\t0x4b, 0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x74, 0x7a,\n\t0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63,\n\t0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e,\n\t0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x54, 0x5a, 0x48, 0x00, 0x52, 0x0c,\n\t0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x74, 0x7a, 0x12, 0x57, 0x0a, 0x11,\n\t0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61,\n\t0x6c, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65,\n\t0x2e, 0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61,\n\t0x6c, 0x48, 0x00, 0x52, 0x10, 0x63, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x49, 0x6e, 0x74,\n\t0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x5b, 0x0a, 0x13, 0x79, 0x65, 0x61, 0x72, 0x5f, 0x6d, 0x6f,\n\t0x6e, 0x74, 0x68, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x12, 0x20, 0x01,\n\t0x28, 0x0b, 0x32, 0x29, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x59, 0x65, 0x61, 0x72,\n\t0x4d, 0x6f, 0x6e, 0x74, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x48, 0x00, 0x52,\n\t0x11, 0x79, 0x65, 0x61, 0x72, 0x4d, 0x6f, 0x6e, 0x74, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76,\n\t0x61, 0x6c, 0x12, 0x55, 0x0a, 0x11, 0x64, 0x61, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x69,\n\t0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e,\n\t0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61,\n\t0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x44, 0x61, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x49, 0x6e,\n\t0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x0f, 0x64, 0x61, 0x79, 0x54, 0x69, 0x6d,\n\t0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x35, 0x0a, 0x05, 0x61, 0x72, 0x72,\n\t0x61, 0x79, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b,\n\t0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70,\n\t0x65, 0x2e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x48, 0x00, 0x52, 0x05, 0x61, 0x72, 0x72, 0x61, 0x79,\n\t0x12, 0x38, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b,\n\t0x32, 0x1e, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,\n\t0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74,\n\t0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x12, 0x2f, 0x0a, 0x03, 0x6d, 0x61,\n\t0x70, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e,\n\t0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65,\n\t0x2e, 0x4d, 0x61, 0x70, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x61, 0x70, 0x12, 0x3b, 0x0a, 0x07, 0x76,\n\t0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74,\n\t0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x56, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x48, 0x00, 0x52,\n\t0x07, 0x76, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x12, 0x2f, 0x0a, 0x03, 0x75, 0x64, 0x74, 0x18,\n\t0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x55,\n\t0x44, 0x54, 0x48, 0x00, 0x52, 0x03, 0x75, 0x64, 0x74, 0x12, 0x3e, 0x0a, 0x08, 0x75, 0x6e, 0x70,\n\t0x61, 0x72, 0x73, 0x65, 0x64, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61,\n\t0x54, 0x79, 0x70, 0x65, 0x2e, 0x55, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x48, 0x00, 0x52,\n\t0x08, 0x75, 0x6e, 0x70, 0x61, 0x72, 0x73, 0x65, 0x64, 0x1a, 0x43, 0x0a, 0x07, 0x42, 0x6f, 0x6f,\n\t0x6c, 0x65, 0x61, 0x6e, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72,\n\t0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x40,\n\t0x0a, 0x04, 0x42, 0x79, 0x74, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76,\n\t0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e,\n\t0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61,\n\t0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65,\n\t0x1a, 0x41, 0x0a, 0x05, 0x53, 0x68, 0x6f, 0x72, 0x74, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70,\n\t0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65,\n\t0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70,\n\t0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65,\n\t0x6e, 0x63, 0x65, 0x1a, 0x43, 0x0a, 0x07, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x12, 0x38,\n\t0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d,\n\t0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,\n\t0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x40, 0x0a, 0x04, 0x4c, 0x6f, 0x6e, 0x67,\n\t0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x41, 0x0a, 0x05, 0x46, 0x6c,\n\t0x6f, 0x61, 0x74, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x42, 0x0a,\n\t0x06, 0x44, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f,\n\t0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65,\n\t0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56,\n\t0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63,\n\t0x65, 0x1a, 0x60, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x38, 0x0a, 0x18, 0x74,\n\t0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65,\n\t0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74,\n\t0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65,\n\t0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6f, 0x6c, 0x6c, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x1a, 0x42, 0x0a, 0x06, 0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x12, 0x38, 0x0a,\n\t0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,\n\t0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52,\n\t0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65,\n\t0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x40, 0x0a, 0x04, 0x4e, 0x55, 0x4c, 0x4c, 0x12,\n\t0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,\n\t0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e,\n\t0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x45, 0x0a, 0x09, 0x54, 0x69, 0x6d,\n\t0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76,\n\t0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e,\n\t0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61,\n\t0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65,\n\t0x1a, 0x40, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x65, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65,\n\t0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72,\n\t0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65,\n\t0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e,\n\t0x63, 0x65, 0x1a, 0x48, 0x0a, 0x0c, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e,\n\t0x54, 0x5a, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01,\n\t0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x4c, 0x0a, 0x10,\n\t0x43, 0x61, 0x6c, 0x65, 0x6e, 0x64, 0x61, 0x72, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c,\n\t0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01,\n\t0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f,\n\t0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0xb3, 0x01, 0x0a, 0x11, 0x59,\n\t0x65, 0x61, 0x72, 0x4d, 0x6f, 0x6e, 0x74, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c,\n\t0x12, 0x24, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x46, 0x69,\n\t0x65, 0x6c, 0x64, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x65, 0x6e, 0x64, 0x5f, 0x66, 0x69,\n\t0x65, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x08, 0x65, 0x6e, 0x64,\n\t0x46, 0x69, 0x65, 0x6c, 0x64, 0x88, 0x01, 0x01, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65,\n\t0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72,\n\t0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65,\n\t0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e,\n\t0x63, 0x65, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x69, 0x65,\n\t0x6c, 0x64, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64,\n\t0x1a, 0xb1, 0x01, 0x0a, 0x0f, 0x44, 0x61, 0x79, 0x54, 0x69, 0x6d, 0x65, 0x49, 0x6e, 0x74, 0x65,\n\t0x72, 0x76, 0x61, 0x6c, 0x12, 0x24, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x66, 0x69,\n\t0x65, 0x6c, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x74, 0x61,\n\t0x72, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x88, 0x01, 0x01, 0x12, 0x20, 0x0a, 0x09, 0x65, 0x6e,\n\t0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52,\n\t0x08, 0x65, 0x6e, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x88, 0x01, 0x01, 0x12, 0x38, 0x0a, 0x18,\n\t0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72,\n\t0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16,\n\t0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66,\n\t0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74,\n\t0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x66,\n\t0x69, 0x65, 0x6c, 0x64, 0x1a, 0x58, 0x0a, 0x04, 0x43, 0x68, 0x61, 0x72, 0x12, 0x16, 0x0a, 0x06,\n\t0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x6c, 0x65,\n\t0x6e, 0x67, 0x74, 0x68, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72,\n\t0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65,\n\t0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69,\n\t0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x5b,\n\t0x0a, 0x07, 0x56, 0x61, 0x72, 0x43, 0x68, 0x61, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x6c, 0x65, 0x6e,\n\t0x67, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x6c, 0x65, 0x6e, 0x67, 0x74,\n\t0x68, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69,\n\t0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x99, 0x01, 0x0a, 0x07,\n\t0x44, 0x65, 0x63, 0x69, 0x6d, 0x61, 0x6c, 0x12, 0x19, 0x0a, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65,\n\t0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x05, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x88,\n\t0x01, 0x01, 0x12, 0x21, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18,\n\t0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69,\n\t0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61,\n\t0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63,\n\t0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72,\n\t0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x42,\n\t0x08, 0x0a, 0x06, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x70, 0x72,\n\t0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0xa1, 0x01, 0x0a, 0x0b, 0x53, 0x74, 0x72, 0x75,\n\t0x63, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18,\n\t0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x34, 0x0a, 0x09, 0x64,\n\t0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17,\n\t0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44,\n\t0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70,\n\t0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20,\n\t0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x1f, 0x0a,\n\t0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48,\n\t0x00, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x88, 0x01, 0x01, 0x42, 0x0b,\n\t0x0a, 0x09, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x7f, 0x0a, 0x06, 0x53,\n\t0x74, 0x72, 0x75, 0x63, 0x74, 0x12, 0x3b, 0x0a, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18,\n\t0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f,\n\t0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x53,\n\t0x74, 0x72, 0x75, 0x63, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c,\n\t0x64, 0x73, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61,\n\t0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x02,\n\t0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74,\n\t0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0xa2, 0x01, 0x0a,\n\t0x05, 0x41, 0x72, 0x72, 0x61, 0x79, 0x12, 0x3a, 0x0a, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,\n\t0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73,\n\t0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74,\n\t0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79,\n\t0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x6e,\n\t0x75, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x61,\n\t0x69, 0x6e, 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f,\n\t0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65,\n\t0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56,\n\t0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63,\n\t0x65, 0x1a, 0xdb, 0x01, 0x0a, 0x03, 0x4d, 0x61, 0x70, 0x12, 0x32, 0x0a, 0x08, 0x6b, 0x65, 0x79,\n\t0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70,\n\t0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61,\n\t0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6b, 0x65, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x36, 0x0a,\n\t0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,\n\t0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,\n\t0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75,\n\t0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x2e, 0x0a, 0x13, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x63,\n\t0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01,\n\t0x28, 0x08, 0x52, 0x11, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e,\n\t0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61,\n\t0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63,\n\t0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79, 0x70, 0x65, 0x56, 0x61, 0x72,\n\t0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x1a,\n\t0x43, 0x0a, 0x07, 0x56, 0x61, 0x72, 0x69, 0x61, 0x6e, 0x74, 0x12, 0x38, 0x0a, 0x18, 0x74, 0x79,\n\t0x70, 0x65, 0x5f, 0x76, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x66,\n\t0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x16, 0x74, 0x79,\n\t0x70, 0x65, 0x56, 0x61, 0x72, 0x69, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x65, 0x72,\n\t0x65, 0x6e, 0x63, 0x65, 0x1a, 0xa1, 0x02, 0x0a, 0x03, 0x55, 0x44, 0x54, 0x12, 0x12, 0x0a, 0x04,\n\t0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65,\n\t0x12, 0x20, 0x0a, 0x09, 0x6a, 0x76, 0x6d, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x02, 0x20,\n\t0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x08, 0x6a, 0x76, 0x6d, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x88,\n\t0x01, 0x01, 0x12, 0x26, 0x0a, 0x0c, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x61,\n\t0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x0b, 0x70, 0x79, 0x74, 0x68,\n\t0x6f, 0x6e, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x88, 0x01, 0x01, 0x12, 0x3b, 0x0a, 0x17, 0x73, 0x65,\n\t0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f,\n\t0x63, 0x6c, 0x61, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x02, 0x52, 0x15, 0x73,\n\t0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x43,\n\t0x6c, 0x61, 0x73, 0x73, 0x88, 0x01, 0x01, 0x12, 0x37, 0x0a, 0x08, 0x73, 0x71, 0x6c, 0x5f, 0x74,\n\t0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x73, 0x70, 0x61, 0x72,\n\t0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x54, 0x79,\n\t0x70, 0x65, 0x48, 0x03, 0x52, 0x07, 0x73, 0x71, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x88, 0x01, 0x01,\n\t0x42, 0x0c, 0x0a, 0x0a, 0x5f, 0x6a, 0x76, 0x6d, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x42, 0x0f,\n\t0x0a, 0x0d, 0x5f, 0x70, 0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x42,\n\t0x1a, 0x0a, 0x18, 0x5f, 0x73, 0x65, 0x72, 0x69, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70,\n\t0x79, 0x74, 0x68, 0x6f, 0x6e, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x42, 0x0b, 0x0a, 0x09, 0x5f,\n\t0x73, 0x71, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x1a, 0x34, 0x0a, 0x08, 0x55, 0x6e, 0x70, 0x61,\n\t0x72, 0x73, 0x65, 0x64, 0x12, 0x28, 0x0a, 0x10, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x79, 0x70,\n\t0x65, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,\n\t0x64, 0x61, 0x74, 0x61, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x06,\n\t0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x42, 0x36, 0x0a, 0x1e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,\n\t0x61, 0x63, 0x68, 0x65, 0x2e, 0x73, 0x70, 0x61, 0x72, 0x6b, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,\n\t0x63, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x12, 0x69, 0x6e, 0x74, 0x65,\n\t0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x64, 0x62, 0x06,\n\t0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,\n}\n\nvar (\n\tfile_spark_connect_types_proto_rawDescOnce sync.Once\n\tfile_spark_connect_types_proto_rawDescData = file_spark_connect_types_proto_rawDesc\n)\n\nfunc file_spark_connect_types_proto_rawDescGZIP() []byte {\n\tfile_spark_connect_types_proto_rawDescOnce.Do(func() {\n\t\tfile_spark_connect_types_proto_rawDescData = protoimpl.X.CompressGZIP(file_spark_connect_types_proto_rawDescData)\n\t})\n\treturn file_spark_connect_types_proto_rawDescData\n}\n\nvar file_spark_connect_types_proto_msgTypes = make([]protoimpl.MessageInfo, 27)\nvar file_spark_connect_types_proto_goTypes = []interface{}{\n\t(*DataType)(nil),                   // 0: spark.connect.DataType\n\t(*DataType_Boolean)(nil),           // 1: spark.connect.DataType.Boolean\n\t(*DataType_Byte)(nil),              // 2: spark.connect.DataType.Byte\n\t(*DataType_Short)(nil),             // 3: spark.connect.DataType.Short\n\t(*DataType_Integer)(nil),           // 4: spark.connect.DataType.Integer\n\t(*DataType_Long)(nil),              // 5: spark.connect.DataType.Long\n\t(*DataType_Float)(nil),             // 6: spark.connect.DataType.Float\n\t(*DataType_Double)(nil),            // 7: spark.connect.DataType.Double\n\t(*DataType_String)(nil),            // 8: spark.connect.DataType.String\n\t(*DataType_Binary)(nil),            // 9: spark.connect.DataType.Binary\n\t(*DataType_NULL)(nil),              // 10: spark.connect.DataType.NULL\n\t(*DataType_Timestamp)(nil),         // 11: spark.connect.DataType.Timestamp\n\t(*DataType_Date)(nil),              // 12: spark.connect.DataType.Date\n\t(*DataType_TimestampNTZ)(nil),      // 13: spark.connect.DataType.TimestampNTZ\n\t(*DataType_CalendarInterval)(nil),  // 14: spark.connect.DataType.CalendarInterval\n\t(*DataType_YearMonthInterval)(nil), // 15: spark.connect.DataType.YearMonthInterval\n\t(*DataType_DayTimeInterval)(nil),   // 16: spark.connect.DataType.DayTimeInterval\n\t(*DataType_Char)(nil),              // 17: spark.connect.DataType.Char\n\t(*DataType_VarChar)(nil),           // 18: spark.connect.DataType.VarChar\n\t(*DataType_Decimal)(nil),           // 19: spark.connect.DataType.Decimal\n\t(*DataType_StructField)(nil),       // 20: spark.connect.DataType.StructField\n\t(*DataType_Struct)(nil),            // 21: spark.connect.DataType.Struct\n\t(*DataType_Array)(nil),             // 22: spark.connect.DataType.Array\n\t(*DataType_Map)(nil),               // 23: spark.connect.DataType.Map\n\t(*DataType_Variant)(nil),           // 24: spark.connect.DataType.Variant\n\t(*DataType_UDT)(nil),               // 25: spark.connect.DataType.UDT\n\t(*DataType_Unparsed)(nil),          // 26: spark.connect.DataType.Unparsed\n}\nvar file_spark_connect_types_proto_depIdxs = []int32{\n\t10, // 0: spark.connect.DataType.null:type_name -> spark.connect.DataType.NULL\n\t9,  // 1: spark.connect.DataType.binary:type_name -> spark.connect.DataType.Binary\n\t1,  // 2: spark.connect.DataType.boolean:type_name -> spark.connect.DataType.Boolean\n\t2,  // 3: spark.connect.DataType.byte:type_name -> spark.connect.DataType.Byte\n\t3,  // 4: spark.connect.DataType.short:type_name -> spark.connect.DataType.Short\n\t4,  // 5: spark.connect.DataType.integer:type_name -> spark.connect.DataType.Integer\n\t5,  // 6: spark.connect.DataType.long:type_name -> spark.connect.DataType.Long\n\t6,  // 7: spark.connect.DataType.float:type_name -> spark.connect.DataType.Float\n\t7,  // 8: spark.connect.DataType.double:type_name -> spark.connect.DataType.Double\n\t19, // 9: spark.connect.DataType.decimal:type_name -> spark.connect.DataType.Decimal\n\t8,  // 10: spark.connect.DataType.string:type_name -> spark.connect.DataType.String\n\t17, // 11: spark.connect.DataType.char:type_name -> spark.connect.DataType.Char\n\t18, // 12: spark.connect.DataType.var_char:type_name -> spark.connect.DataType.VarChar\n\t12, // 13: spark.connect.DataType.date:type_name -> spark.connect.DataType.Date\n\t11, // 14: spark.connect.DataType.timestamp:type_name -> spark.connect.DataType.Timestamp\n\t13, // 15: spark.connect.DataType.timestamp_ntz:type_name -> spark.connect.DataType.TimestampNTZ\n\t14, // 16: spark.connect.DataType.calendar_interval:type_name -> spark.connect.DataType.CalendarInterval\n\t15, // 17: spark.connect.DataType.year_month_interval:type_name -> spark.connect.DataType.YearMonthInterval\n\t16, // 18: spark.connect.DataType.day_time_interval:type_name -> spark.connect.DataType.DayTimeInterval\n\t22, // 19: spark.connect.DataType.array:type_name -> spark.connect.DataType.Array\n\t21, // 20: spark.connect.DataType.struct:type_name -> spark.connect.DataType.Struct\n\t23, // 21: spark.connect.DataType.map:type_name -> spark.connect.DataType.Map\n\t24, // 22: spark.connect.DataType.variant:type_name -> spark.connect.DataType.Variant\n\t25, // 23: spark.connect.DataType.udt:type_name -> spark.connect.DataType.UDT\n\t26, // 24: spark.connect.DataType.unparsed:type_name -> spark.connect.DataType.Unparsed\n\t0,  // 25: spark.connect.DataType.StructField.data_type:type_name -> spark.connect.DataType\n\t20, // 26: spark.connect.DataType.Struct.fields:type_name -> spark.connect.DataType.StructField\n\t0,  // 27: spark.connect.DataType.Array.element_type:type_name -> spark.connect.DataType\n\t0,  // 28: spark.connect.DataType.Map.key_type:type_name -> spark.connect.DataType\n\t0,  // 29: spark.connect.DataType.Map.value_type:type_name -> spark.connect.DataType\n\t0,  // 30: spark.connect.DataType.UDT.sql_type:type_name -> spark.connect.DataType\n\t31, // [31:31] is the sub-list for method output_type\n\t31, // [31:31] is the sub-list for method input_type\n\t31, // [31:31] is the sub-list for extension type_name\n\t31, // [31:31] is the sub-list for extension extendee\n\t0,  // [0:31] is the sub-list for field type_name\n}\n\nfunc init() { file_spark_connect_types_proto_init() }\nfunc file_spark_connect_types_proto_init() {\n\tif File_spark_connect_types_proto != nil {\n\t\treturn\n\t}\n\tif !protoimpl.UnsafeEnabled {\n\t\tfile_spark_connect_types_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Boolean); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Byte); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Short); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Integer); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Long); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Float); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Double); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_String); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Binary); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_NULL); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Timestamp); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Date); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_TimestampNTZ); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_CalendarInterval); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_YearMonthInterval); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_DayTimeInterval); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Char); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_VarChar); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Decimal); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_StructField); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Struct); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Array); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Map); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Variant); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_UDT); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t\tfile_spark_connect_types_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {\n\t\t\tswitch v := v.(*DataType_Unparsed); i {\n\t\t\tcase 0:\n\t\t\t\treturn &v.state\n\t\t\tcase 1:\n\t\t\t\treturn &v.sizeCache\n\t\t\tcase 2:\n\t\t\t\treturn &v.unknownFields\n\t\t\tdefault:\n\t\t\t\treturn nil\n\t\t\t}\n\t\t}\n\t}\n\tfile_spark_connect_types_proto_msgTypes[0].OneofWrappers = []interface{}{\n\t\t(*DataType_Null)(nil),\n\t\t(*DataType_Binary_)(nil),\n\t\t(*DataType_Boolean_)(nil),\n\t\t(*DataType_Byte_)(nil),\n\t\t(*DataType_Short_)(nil),\n\t\t(*DataType_Integer_)(nil),\n\t\t(*DataType_Long_)(nil),\n\t\t(*DataType_Float_)(nil),\n\t\t(*DataType_Double_)(nil),\n\t\t(*DataType_Decimal_)(nil),\n\t\t(*DataType_String_)(nil),\n\t\t(*DataType_Char_)(nil),\n\t\t(*DataType_VarChar_)(nil),\n\t\t(*DataType_Date_)(nil),\n\t\t(*DataType_Timestamp_)(nil),\n\t\t(*DataType_TimestampNtz)(nil),\n\t\t(*DataType_CalendarInterval_)(nil),\n\t\t(*DataType_YearMonthInterval_)(nil),\n\t\t(*DataType_DayTimeInterval_)(nil),\n\t\t(*DataType_Array_)(nil),\n\t\t(*DataType_Struct_)(nil),\n\t\t(*DataType_Map_)(nil),\n\t\t(*DataType_Variant_)(nil),\n\t\t(*DataType_Udt)(nil),\n\t\t(*DataType_Unparsed_)(nil),\n\t}\n\tfile_spark_connect_types_proto_msgTypes[15].OneofWrappers = []interface{}{}\n\tfile_spark_connect_types_proto_msgTypes[16].OneofWrappers = []interface{}{}\n\tfile_spark_connect_types_proto_msgTypes[19].OneofWrappers = []interface{}{}\n\tfile_spark_connect_types_proto_msgTypes[20].OneofWrappers = []interface{}{}\n\tfile_spark_connect_types_proto_msgTypes[25].OneofWrappers = []interface{}{}\n\ttype x struct{}\n\tout := protoimpl.TypeBuilder{\n\t\tFile: protoimpl.DescBuilder{\n\t\t\tGoPackagePath: reflect.TypeOf(x{}).PkgPath(),\n\t\t\tRawDescriptor: file_spark_connect_types_proto_rawDesc,\n\t\t\tNumEnums:      0,\n\t\t\tNumMessages:   27,\n\t\t\tNumExtensions: 0,\n\t\t\tNumServices:   0,\n\t\t},\n\t\tGoTypes:           file_spark_connect_types_proto_goTypes,\n\t\tDependencyIndexes: file_spark_connect_types_proto_depIdxs,\n\t\tMessageInfos:      file_spark_connect_types_proto_msgTypes,\n\t}.Build()\n\tFile_spark_connect_types_proto = out.File\n\tfile_spark_connect_types_proto_rawDesc = nil\n\tfile_spark_connect_types_proto_goTypes = nil\n\tfile_spark_connect_types_proto_depIdxs = nil\n}\n"
  },
  {
    "path": "internal/tests/integration/dataframe_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage integration\n\nimport (\n\t\"context\"\n\t\"fmt\"\n\t\"os\"\n\t\"testing\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/column\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n\t\"github.com/stretchr/testify/assert\"\n\t\"github.com/stretchr/testify/require\"\n)\n\nfunc TestDataFrame_Select(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tassert.NoError(t, err)\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tdf, err = df.Select(ctx, functions.StringLit(\"1\"), functions.StringLit(\"2\"))\n\tassert.NoError(t, err)\n\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 100, len(res))\n\n\trowZero := res[0]\n\tassert.Equal(t, 2, rowZero.Len())\n\n\tdf, err = spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\t_, err = df.Select(ctx, column.OfDF(df, \"id2\"))\n\tassert.Error(t, err)\n}\n\nfunc TestDataFrame_SelectExpr(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tassert.NoError(t, err)\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tdf, err = df.SelectExpr(ctx, \"1\", \"2\", \"spark_partition_id()\")\n\tassert.NoError(t, err)\n\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 100, len(res))\n\n\trow_zero := res[0]\n\tassert.Equal(t, 3, row_zero.Len())\n}\n\nfunc TestDataFrame_Alias(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tassert.NoError(t, err)\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tdf = df.Alias(ctx, \"df\")\n\tres, er := df.Collect(ctx)\n\tassert.NoError(t, er)\n\tassert.Equal(t, 100, len(res))\n}\n\nfunc TestDataFrame_CrossJoin(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tassert.NoError(t, err)\n\tdf1, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf2, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf := df1.CrossJoin(ctx, df2)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 100, len(res))\n\tassert.Equal(t, 2, res[0].Len())\n}\n\nfunc TestDataFrame_GroupBy(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, _ := spark.Sql(ctx, \"select 'a' as a, 1 as b from range(10)\")\n\tdf, _ := src.GroupBy(functions.Col(\"a\")).Agg(ctx, functions.Sum(functions.Col(\"b\")))\n\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(res))\n\n\tdf, err = src.GroupBy(functions.Col(\"a\")).Count(ctx)\n\tassert.NoError(t, err)\n\tres, err = df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(res))\n\tassert.Equal(t, \"a\", res[0].At(0))\n\tassert.Equal(t, int64(10), res[0].At(1))\n}\n\nfunc TestDataFrame_Count(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, _ := spark.Sql(ctx, \"select 'a' as a, 1 as b from range(10)\")\n\tres, err := src.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(10), res)\n}\n\nfunc TestDataFrame_OfDFWithRegex(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, _ := spark.Sql(ctx, \"select 'a' as myColumnName, 1 as b from range(10)\")\n\tcol := column.OfDFWithRegex(src, \"`.*(Column).*`\")\n\tres, err := src.Select(ctx, col)\n\tassert.NoError(t, err)\n\tschema, err := res.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n}\n\nfunc TestSparkSession_CreateDataFrame(t *testing.T) {\n\tctx, spark := connect()\n\n\ttbl := createArrowTable()\n\tdefer tbl.Release()\n\n\tdf, err := spark.CreateDataFrameFromArrow(ctx, tbl)\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 3, len(res))\n}\n\nfunc TestSparkSession_CreateDataFrameWithSchema(t *testing.T) {\n\tctx, spark := connect()\n\n\tdata := [][]any{\n\t\t{1, 1.1, \"a\"},\n\t\t{2, 2.2, \"b\"},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"f1-i32\", types.INTEGER),\n\t\ttypes.NewStructField(\"f2-f64\", types.DOUBLE),\n\t\ttypes.NewStructField(\"f3-string\", types.STRING))\n\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 2, len(res))\n\tassert.Equal(t, 3, res[0].Len())\n\tassert.Equal(t, int32(1), res[0].At(0))\n\tassert.Equal(t, 1.1, res[0].At(1))\n\tassert.Equal(t, \"a\", res[0].At(2))\n}\n\nfunc TestDataFrame_Corr(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{\n\t\t{1, 12}, {10, 1}, {19, 8},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"c1\", types.INTEGER),\n\t\ttypes.NewStructField(\"c2\", types.INTEGER),\n\t)\n\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tres, err := df.Corr(ctx, \"c1\", \"c2\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, -0.3592106040535498, res)\n\n\tres2, err := df.Stat().Corr(ctx, \"c1\", \"c2\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, res, res2)\n}\n\nfunc TestDataFrame_Cov(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{\n\t\t{1, 12}, {10, 1}, {19, 8},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"c1\", types.INTEGER),\n\t\ttypes.NewStructField(\"c2\", types.INTEGER),\n\t)\n\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tres, err := df.Cov(ctx, \"c1\", \"c2\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, -18.0, res)\n\n\tres2, err := df.Stat().Cov(ctx, \"c1\", \"c2\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, res, res2)\n}\n\nfunc TestDataFrame_WithColumn(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithColumn(ctx, \"newCol\", functions.IntLit(1))\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 10, len(res))\n\t// Check the values of the new column\n\tfor _, row := range res {\n\t\tassert.Equal(t, 2, row.Len())\n\t\tassert.Equal(t, int64(1), row.At(1))\n\t}\n}\n\nfunc TestDataFrame_WithColumns(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithColumns(ctx, column.WithAlias(\"newCol1\", functions.IntLit(1)),\n\t\tcolumn.WithAlias(\"newCol2\", functions.IntLit(2)))\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 10, len(res))\n\t// Check the values of the new columns\n\tfor _, row := range res {\n\t\tassert.Equal(t, 3, row.Len())\n\t\tassert.Equal(t, int64(1), row.At(1))\n\t\tassert.Equal(t, int64(2), row.At(2))\n\t}\n}\n\nfunc TestDataFrame_WithMetadata(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithMetadata(ctx, map[string]string{\"id\": \"value\"})\n\tassert.NoError(t, err)\n\t_, err = df.Schema(ctx)\n\tassert.Error(t, err, \"Expecting malformed metadata\")\n\n\tdf, err = spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithMetadata(ctx, map[string]string{\"id\": \"{\\\"kk\\\": \\\"value\\\"}\"})\n\tassert.NoError(t, err)\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tfields := schema.Fields[0]\n\tassert.Equal(t, \"id\", fields.Name)\n\tassert.Equal(t, \"{\\\"kk\\\":\\\"value\\\"}\", *fields.Metadata)\n}\n\nfunc TestDataFrame_WithColumnRenamed(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithColumnRenamed(ctx, \"id\", \"newId\")\n\tassert.NoError(t, err)\n\t// Check the schema of the new dataframe\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"newId\", schema.Fields[0].Name)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 10, len(res))\n\t// Check the values of the new column\n\tfor i, row := range res {\n\t\tassert.Equal(t, 1, row.Len())\n\t\tassert.Equal(t, int64(i), row.At(0))\n\t}\n\n\t// Test that renaming a non-existing column does not change anything.\n\tdf, _ = spark.Sql(ctx, \"select * from range(10)\")\n\tdf, err = df.WithColumnRenamed(ctx, \"nonExisting\", \"newId\")\n\tassert.NoError(t, err)\n\tschema, err = df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"id\", schema.Fields[0].Name)\n\n\t// Test that single column renaming works as well.\n\tdf, _ = spark.Sql(ctx, \"select * from range(10)\")\n\tdf, err = df.WithColumnRenamed(ctx, \"id\", \"newId\")\n\tassert.NoError(t, err)\n\tschema, err = df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"newId\", schema.Fields[0].Name)\n}\n\nfunc TestDataFrame_WithWatermark(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select current_timestamp() as this_time from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.WithWatermark(ctx, \"this_time\", \"1 minute\")\n\tassert.NoError(t, err)\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"this_time\", schema.Fields[0].Name)\n}\n\nfunc TestDataFrame_Where(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err = df.Where(ctx, \"id = 0\")\n\tassert.NoError(t, err)\n\tres, err := df.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(1), res)\n}\n\nfunc TestDataFrame_Drop(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, err := spark.Sql(ctx, \"select 1 as id, 2 as other from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err := src.DropByName(ctx, \"id\")\n\tassert.NoError(t, err)\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"other\", schema.Fields[0].Name)\n\n\tdf, err = src.Drop(ctx, column.OfDF(src, \"other\"))\n\tassert.NoError(t, err)\n\tschema, err = df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(schema.Fields))\n\tassert.Equal(t, \"id\", schema.Fields[0].Name)\n}\n\nfunc TestDataFrame_DropDuplicates(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, err := spark.Sql(ctx, \"select 1 as id, 2 as other from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err := src.DropDuplicates(ctx)\n\tassert.NoError(t, err)\n\tres, err := df.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(1), res)\n\n\t// Create a dataframe with duplicate rows\n\tdata := [][]any{\n\t\t{\"Alice\", 5, 80}, {\"Alice\", 5, 80}, {\"Alice\", 10, 80},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"name\", types.STRING),\n\t\ttypes.NewStructField(\"age\", types.INTEGER),\n\t\ttypes.NewStructField(\"height\", types.INTEGER),\n\t)\n\n\tdf, err = spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\t// Check the schema of the dataframe\n\tschema, err = df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 3, len(schema.Fields))\n\tassert.Equal(t, \"name\", schema.Fields[0].Name)\n\tassert.Equal(t, \"age\", schema.Fields[1].Name)\n\tassert.Equal(t, \"height\", schema.Fields[2].Name)\n\n\tdf, err = df.DropDuplicates(ctx)\n\tassert.NoError(t, err)\n\tres, err = df.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(2), res)\n\t// Check the two ages\n\trows, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 2, len(rows))\n\tassert.Equal(t, \"Alice\", rows[0].At(0))\n\tassert.Equal(t, int32(5), rows[0].At(1))\n\tassert.Equal(t, \"Alice\", rows[1].At(0))\n\tassert.Equal(t, int32(10), rows[1].At(1))\n\n\t// Test drop duplicates with column names\n\tdf, err = df.DropDuplicates(ctx, \"name\")\n\tassert.NoError(t, err)\n\tres, err = df.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(1), res)\n}\n\nfunc TestDataFrame_Explain(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tres, err := df.Explain(ctx, utils.ExplainModeSimple)\n\tassert.NoError(t, err)\n\tassert.Contains(t, res, \"Physical Plan\")\n\n\tres, err = df.Explain(ctx, utils.ExplainModeExtended)\n\tassert.NoError(t, err)\n\tassert.Contains(t, res, \"Physical Plan\")\n\n\tres, err = df.Explain(ctx, utils.ExplainModeCodegen)\n\tassert.NoError(t, err)\n\tassert.Contains(t, res, \"WholeStageCodegen\")\n\n\tres, err = df.Explain(ctx, utils.ExplainModeCost)\n\tassert.NoError(t, err)\n\tassert.Contains(t, res, \"Physical Plan\")\n\n\tres, err = df.Explain(ctx, utils.ExplainModeFormatted)\n\tassert.NoError(t, err)\n\tassert.Contains(t, res, \"Physical Plan\")\n}\n\nfunc TestDataFrame_CachingAndPersistence(t *testing.T) {\n\tctx, spark := connect()\n\tlevels := []utils.StorageLevel{\n\t\tutils.StorageLevelDiskOnly,\n\t\tutils.StorageLevelDiskOnly2,\n\t\tutils.StorageLevelDiskOnly3,\n\t\tutils.StorageLevelMemoryAndDisk,\n\t\tutils.StorageLevelMemoryAndDisk2,\n\t\tutils.StorageLevelMemoryOnly,\n\t\tutils.StorageLevelMemoryOnly2,\n\t\tutils.StorageLevelMemoyAndDiskDeser,\n\t\tutils.StorageLevelOffHeap,\n\t}\n\n\tfor _, lvl := range levels {\n\t\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\t\tassert.NoError(t, err)\n\t\terr = df.Persist(ctx, lvl)\n\t\tassert.NoError(t, err)\n\t\tl, err := df.GetStorageLevel(ctx)\n\t\tassert.NoError(t, err)\n\n\t\tassert.Contains(t, []utils.StorageLevel{lvl, utils.StorageLevelMemoryOnly}, *l)\n\n\t\terr = df.Unpersist(ctx)\n\t\tassert.NoError(t, err)\n\t}\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\terr = df.Cache(ctx)\n\tassert.NoError(t, err)\n\tl, err := df.GetStorageLevel(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, utils.StorageLevelMemoryOnly, *l, \"%v != %v\", utils.StorageLevelMemoryOnly, *l)\n}\n\nfunc TestDataFrame_SetOps(t *testing.T) {\n\tctx, spark := connect()\n\tdf1, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf2, err := spark.Sql(ctx, \"select * from range(5)\")\n\tassert.NoError(t, err)\n\n\tdf := df1.Union(ctx, df2)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 15, len(res))\n\n\tdf = df1.Intersect(ctx, df2)\n\tres, err = df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 5, len(res))\n\n\tdf = df1.ExceptAll(ctx, df2)\n\tres, err = df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 5, len(res))\n}\n\nfunc TestDataFrame_ToArrow(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\ttbl, err := df.ToArrow(ctx)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, tbl)\n}\n\nfunc TestDataFrame_LimitVersions(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf = df.Limit(ctx, int32(5))\n\tassert.NoError(t, err)\n\trows, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 5)\n\n\trows, err = df.Tail(ctx, int32(3))\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 3)\n\n\trows, err = df.Head(ctx, int32(3))\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 3)\n\n\trows, err = df.Take(ctx, int32(3))\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 3)\n}\n\nfunc TestDataFrame_Sort(t *testing.T) {\n\tctx, spark := connect()\n\tsrc, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf, err := src.Sort(ctx, functions.Col(\"id\").Desc())\n\tassert.NoError(t, err)\n\tres, err := df.Head(ctx, 1)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(9), res[0].At(0))\n\n\tdf, err = src.Sort(ctx, functions.Col(\"id\").Asc())\n\tassert.NoError(t, err)\n\tres, err = df.Head(ctx, 1)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(0), res[0].At(0))\n}\n\nfunc TestDataFrame_Join(t *testing.T) {\n\tctx, spark := connect()\n\tdf1, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf2, err := spark.Sql(ctx, \"select * from range(5)\")\n\tassert.NoError(t, err)\n\n\tdf, err := df1.Join(ctx, df2, column.OfDF(df1, \"id\").Eq(column.OfDF(df2, \"id\")), utils.JoinTypeInner)\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 5, len(res))\n}\n\nfunc TestDataFrame_RandomSplits(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(1000)\")\n\tassert.NoError(t, err)\n\tdfs, err := df.RandomSplit(ctx, []float64{0.3, 0.7})\n\tassert.NoError(t, err)\n\tassert.Len(t, dfs, 2)\n\tc1, err := dfs[0].Count(ctx)\n\tassert.NoError(t, err)\n\tc2, err := dfs[1].Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Less(t, c1, c2)\n}\n\nfunc TestDataFrame_Describe(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tres, err := df.Describe(ctx, \"id\").Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 5)\n}\n\nfunc TestDataFrame_Summary(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select id, 'a' as col, 2 as other from range(10)\")\n\tassert.NoError(t, err)\n\tres, err := df.Summary(ctx, \"count\", \"stddev\").Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 2)\n\tassert.Equal(t, \"count\", res[0].At(0))\n\tassert.Equal(t, 4, res[0].Len())\n}\n\nfunc TestDataFrame_Pivot(t *testing.T) {\n\tctx, spark := connect()\n\n\tdata := [][]any{\n\t\t{\"dotNET\", 2012, 10000},\n\t\t{\"Java\", 2012, 20000},\n\t\t{\"dotNET\", 2012, 5000},\n\t\t{\"dotNET\", 2013, 48000},\n\t\t{\"Java\", 2013, 30000},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"course\", types.STRING),\n\t\ttypes.NewStructField(\"year\", types.INTEGER),\n\t\ttypes.NewStructField(\"earnings\", types.INTEGER))\n\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tgd := df.GroupBy(functions.Col(\"year\"))\n\tgd, err = gd.Pivot(ctx, \"course\", []types.LiteralType{types.String(\"Java\"), types.String(\"dotNET\")})\n\tassert.NoError(t, err)\n\tdf, err = gd.Sum(ctx, \"earnings\")\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 2)\n}\n\nfunc TestDataFrame_Offset(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf = df.Offset(ctx, int32(5))\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 5)\n}\n\nfunc TestDataFrame_IsEmpty(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tempty, err := df.IsEmpty(ctx)\n\tassert.NoError(t, err)\n\tassert.False(t, empty)\n\n\tdf, err = spark.Sql(ctx, \"select * from range(0)\")\n\tassert.NoError(t, err)\n\tempty, err = df.IsEmpty(ctx)\n\tassert.NoError(t, err)\n\tassert.True(t, empty)\n}\n\nfunc TestDataFrame_First(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\trow, err := df.First(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(0), row.At(0))\n}\n\nfunc TestDataFrame_Distinct(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf = df.Distinct(ctx)\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 10)\n}\n\nfunc TestDataFrame_CrossTab(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{{1, 11}, {1, 11}, {3, 10}, {4, 8}, {4, 8}}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"c1\", types.INTEGER),\n\t\ttypes.NewStructField(\"c2\", types.INTEGER),\n\t)\n\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tdf = df.CrossTab(ctx, \"c1\", \"c2\")\n\tdf, err = df.Sort(ctx, column.OfDF(df, \"c1_c2\").Asc())\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 3)\n\tassert.Equal(t, \"1\", res[0].At(0))\n\tassert.Equal(t, int64(0), res[0].At(1))\n\tassert.Equal(t, int64(2), res[0].At(2))\n\tassert.Equal(t, int64(0), res[0].At(3))\n\n\tdf, err = spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tdf = df.Stat().CrossTab(ctx, \"c1\", \"c2\")\n\tdf, err = df.Sort(ctx, column.OfDF(df, \"c1_c2\").Asc())\n\tassert.NoError(t, err)\n\tres2, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\n\tassert.Equal(t, res, res2)\n}\n\nfunc TestDataFrame_SameSemantics(t *testing.T) {\n\tctx, spark := connect()\n\tdf1, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tdf2, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tres, _ := df1.SameSemantics(ctx, df2)\n\tassert.True(t, res)\n}\n\nfunc TestDataFrame_SemanticHash(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\thash, err := df.SemanticHash(ctx)\n\tassert.NoError(t, err)\n\tassert.NotEmpty(t, hash)\n}\n\nfunc TestDataFrame_FreqItems(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select id % 4 as id from range(100)\")\n\tassert.NoError(t, err)\n\tres, err := df.FreqItems(ctx, \"id\").Collect(ctx)\n\tassert.NoErrorf(t, err, \"%+v\", err)\n\tassert.Len(t, res, 1)\n\n\tres2, err := df.Stat().FreqItems(ctx, \"id\").Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, res, res2)\n}\n\nfunc TestDataFrame_Config_GetAll(t *testing.T) {\n\tctx, spark := connect()\n\tresult, err := spark.Config().GetAll(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, \"driver\", result[\"spark.executor.id\"])\n}\n\nfunc TestDataFrame_Config_Get(t *testing.T) {\n\tctx, spark := connect()\n\tresult, err := spark.Config().Get(ctx, \"spark.executor.id\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, \"driver\", result)\n}\n\nfunc TestDataFrame_Config_GetWithDefault(t *testing.T) {\n\tctx, spark := connect()\n\n\tresult, err := spark.Config().GetWithDefault(ctx, \"spark.whatever\", \"whatever_not_set\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, \"whatever_not_set\", result)\n}\n\nfunc TestDataFrame_Config_Set(t *testing.T) {\n\tctx, spark := connect()\n\terr := spark.Config().Set(ctx, \"spark.whatever\", \"whatever_set\")\n\tassert.NoError(t, err)\n}\n\nfunc TestDataFrame_Config_IsModifiable(t *testing.T) {\n\tctx, spark := connect()\n\tresult, err := spark.Config().IsModifiable(ctx, \"spark.executor.id\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, false, result)\n}\n\nfunc TestDataFrame_Config_Unset(t *testing.T) {\n\tctx, spark := connect()\n\terr := spark.Config().Set(ctx, \"spark.whatever\", \"whatever_set\")\n\tassert.NoError(t, err)\n\terr = spark.Config().Unset(ctx, \"spark.whatever\")\n\tassert.NoError(t, err)\n}\n\nfunc TestDataFrame_Config_e2e_test(t *testing.T) {\n\tctx, spark := connect()\n\t//  add keys that we know is \"modifiable\"\n\tkey := \"spark.sql.ansi.enabled\"\n\tresult, err := spark.Config().IsModifiable(ctx, key)\n\tassert.NoError(t, err)\n\tassert.Equal(t, true, result)\n\t_, err = spark.Config().Get(ctx, key)\n\tassert.NoError(t, err)\n\terr = spark.Config().Set(ctx, \"spark.sql.ansi.enabled\", \"true\")\n\tassert.NoError(t, err)\n\tm, err := spark.Config().Get(ctx, \"spark.sql.ansi.enabled\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, \"true\", m)\n}\n\nfunc TestDataFrame_WithOption(t *testing.T) {\n\tctx, spark := connect()\n\tfile, err := os.CreateTemp(\"\", \"example\")\n\tdefer os.Remove(file.Name())\n\tassert.NoError(t, err)\n\tdefer file.Close()\n\t_, err = file.WriteString(\"id#name,name\\n\")\n\tassert.NoError(t, err)\n\tfor i := 0; i < 10; i++ {\n\t\t_, err = fmt.Fprintf(file, \"%d#alice,alice\\n\", i)\n\t\tassert.NoError(t, err)\n\t}\n\tdf, err := spark.Read().Format(\"csv\").\n\t\tOption(\"header\", \"true\").\n\t\tOption(\"quote\", \"\\\"\").\n\t\tOption(\"sep\", \"#\").\n\t\tOption(\"escapeQuotes\", \"true\").\n\t\t// Option(\"skipLines\", \"5\"). //TODO: this needs more insight\n\t\tOption(\"inferSchema\", \"false\").\n\t\tLoad(file.Name())\n\tassert.NoError(t, err)\n\tc, err := df.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(10), c)\n}\n\nfunc TestDataFrame_Sample(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\ttestCases := []struct {\n\t\tname     string\n\t\tfraction float64\n\t}{\n\t\t{\n\t\t\tname:     \"Default behavior\",\n\t\t\tfraction: 0.1,\n\t\t},\n\t\t{\n\t\t\tname:     \"Large fraction\",\n\t\t\tfraction: 0.9,\n\t\t},\n\t}\n\tfor _, tc := range testCases {\n\t\tt.Run(tc.name, func(t *testing.T) {\n\t\t\tsampledDF, err := df.Sample(ctx, tc.fraction)\n\t\t\tassert.NoError(t, err)\n\t\t\tcount, err := sampledDF.Count(ctx)\n\t\t\tassert.NoError(t, err)\n\t\t\texpectedSize := int(100 * tc.fraction)\n\t\t\tassert.InDelta(t, expectedSize, count, float64(expectedSize), 10)\n\t\t\trows, err := sampledDF.Collect(ctx)\n\t\t\tassert.NoError(t, err)\n\t\t\t// If sampling without replacement, check for duplicates\n\t\t\tseen := make(map[int64]bool)\n\t\t\tfor _, row := range rows {\n\t\t\t\tvalue := row.At(0).(int64)\n\t\t\t\tif seen[value] {\n\t\t\t\t\tt.Fatal(\"Found duplicate value when sampling without replacement\")\n\t\t\t\t}\n\t\t\t\tseen[value] = true\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestDataFrame_SampleWithReplacement(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\ttestCases := []struct {\n\t\tname            string\n\t\twithReplacement bool\n\t\tfraction        float64\n\t}{\n\t\t{\n\t\t\tname:            \"With replacement\",\n\t\t\twithReplacement: true,\n\t\t\tfraction:        0.1,\n\t\t},\n\t\t{\n\t\t\tname:            \"Without replacement\",\n\t\t\twithReplacement: false,\n\t\t\tfraction:        0.1,\n\t\t},\n\t}\n\tfor _, tc := range testCases {\n\t\tt.Run(tc.name, func(t *testing.T) {\n\t\t\tsampledDF, err := df.SampleWithReplacement(ctx, tc.withReplacement, tc.fraction)\n\t\t\tassert.NoError(t, err)\n\t\t\tcount, err := sampledDF.Count(ctx)\n\t\t\tassert.NoError(t, err)\n\t\t\texpectedSize := int(100 * tc.fraction)\n\t\t\tassert.InDelta(t, expectedSize, count, float64(expectedSize), 10)\n\t\t\trows, err := sampledDF.Collect(ctx)\n\t\t\tassert.NoError(t, err)\n\t\t\t// If sampling without replacement, check for duplicates\n\t\t\tif tc.withReplacement == false {\n\t\t\t\tseen := make(map[int64]bool)\n\t\t\t\tfor _, row := range rows {\n\t\t\t\t\tvalue := row.At(0).(int64)\n\t\t\t\t\tif seen[value] {\n\t\t\t\t\t\tt.Fatal(\"Found duplicate value when sampling without replacement\")\n\t\t\t\t\t}\n\t\t\t\t\tseen[value] = true\n\t\t\t\t}\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestDataFrame_SampleSeed(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tfraction := 0.1\n\tseed := int64(17)\n\tsampledDF, err := df.SampleWithSeed(ctx, fraction, seed)\n\tassert.NoError(t, err)\n\tcount, err := sampledDF.Count(ctx)\n\tassert.NoError(t, err)\n\texpectedSize := int(100 * fraction)\n\tassert.InDelta(t, expectedSize, count, float64(expectedSize), 10)\n\trows, err := sampledDF.Collect(ctx)\n\tassert.NoError(t, err)\n\t// If sampling without replacement, check for duplicates\n\tseen := make(map[int64]bool)\n\tfor _, row := range rows {\n\t\tvalue := row.At(0).(int64)\n\t\tif seen[value] {\n\t\t\tt.Fatal(\"Found duplicate value when sampling without replacement\")\n\t\t}\n\t\tseen[value] = true\n\t}\n\t// same seed should return same output\n\tsampledDFRepeat, err := df.SampleWithSeed(ctx, fraction, seed)\n\tassert.NoError(t, err)\n\tcount2, err := sampledDFRepeat.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, count, count2)\n\trows2, err := sampledDFRepeat.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, rows, rows2)\n}\n\nfunc TestDataFrame_SampleWithReplacementSeed(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tfraction := 0.1\n\tseed := int64(17)\n\tsampledDF, err := df.SampleWithReplacementAndSeed(ctx, true, fraction, seed)\n\tassert.NoError(t, err)\n\tcount, err := sampledDF.Count(ctx)\n\tassert.NoError(t, err)\n\texpectedSize := int(100 * fraction)\n\tassert.InDelta(t, expectedSize, count, float64(expectedSize), 10)\n\trows, err := sampledDF.Collect(ctx)\n\tassert.NoError(t, err)\n\t// same seed should return same output\n\tsampledDFRepeat, err := df.SampleWithReplacementAndSeed(ctx, true, fraction, seed)\n\tassert.NoError(t, err)\n\tcount2, err := sampledDFRepeat.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, count, count2)\n\trows2, err := sampledDFRepeat.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, rows, rows2)\n}\n\nfunc TestDataFrame_Unpivot(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{{1, 11, 1.1}, {2, 12, 1.2}}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"id\", types.INTEGER),\n\t\ttypes.NewStructField(\"int\", types.INTEGER),\n\t\ttypes.NewStructField(\"double\", types.DOUBLE),\n\t)\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\n\tudf, err := df.Unpivot(ctx, []column.Convertible{functions.Col(\"id\")},\n\t\t[]column.Convertible{functions.Col(\"int\"), functions.Col(\"double\")},\n\t\t\"type\", \"value\")\n\n\tassert.NoError(t, err)\n\tcnt, err := udf.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(4), cnt)\n}\n\nfunc TestDataFrame_Replace(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{\n\t\t{10, 80, \"Alice\"},\n\t\t{5, nil, \"Bob\"},\n\t\t{nil, 10, \"Tom\"},\n\t\t{nil, nil, nil},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"age\", types.INTEGER),\n\t\ttypes.NewStructField(\"height\", types.INTEGER),\n\t\ttypes.NewStructField(\"name\", types.STRING),\n\t)\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\n\tres, err := df.Replace(ctx,\n\t\t[]types.PrimitiveTypeLiteral{types.Int32(10)},\n\t\t[]types.PrimitiveTypeLiteral{types.Int32(20)},\n\t)\n\tassert.NoError(t, err)\n\n\tcnt, err := res.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(4), cnt)\n\n\trows, err := res.Collect(ctx)\n\tassert.NoError(t, err)\n\n\tassert.Equal(t, int32(20), rows[0].At(0))\n\tassert.Equal(t, int32(20), rows[2].At(1))\n\n\tres, err = df.Replace(ctx,\n\t\t[]types.PrimitiveTypeLiteral{types.Int32(10)},\n\t\t[]types.PrimitiveTypeLiteral{types.Int32Nil},\n\t)\n\tassert.NoError(t, err)\n\n\trows, err = res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Nil(t, rows[0].At(0))\n}\n\nfunc TestDataFrame_ReplaceWithColumn(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{\n\t\t{10, 80, \"Alice\"},\n\t\t{5, nil, \"Bob\"},\n\t\t{nil, 10, \"Tom\"},\n\t\t{nil, nil, nil},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"age\", types.INTEGER),\n\t\ttypes.NewStructField(\"height\", types.INTEGER),\n\t\ttypes.NewStructField(\"name\", types.STRING),\n\t)\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\n\tres, err := df.Replace(ctx, []types.PrimitiveTypeLiteral{types.Int32(10)},\n\t\t[]types.PrimitiveTypeLiteral{types.Int32(20)}, \"age\")\n\tassert.NoError(t, err)\n\n\trows, err := res.Collect(ctx)\n\tassert.NoError(t, err)\n\t// Should only repalce the age column but not the height column\n\tassert.Equal(t, int32(20), rows[0].At(0))\n\tassert.Equal(t, int32(10), rows[2].At(1))\n}\n\nfunc TestDataFrame_FillNa(t *testing.T) {\n\tctx, spark := connect()\n\tfile, err := os.CreateTemp(\"\", \"fillna\")\n\tdefer os.Remove(file.Name())\n\tassert.NoError(t, err)\n\tdefer file.Close()\n\t_, err = file.WriteString(`{\"id\":1,\"int\":null, \"int2\": 1}\n{\"id\":null,\"int\":12, \"int2\": null}\n`)\n\tassert.NoError(t, err)\n\n\tdf, err := spark.Read().Format(\"json\").\n\t\tOption(\"inferSchema\", \"true\").\n\t\tLoad(file.Name())\n\tassert.NoError(t, err)\n\n\t// all columns\n\tfilled, err := df.FillNa(ctx, types.Int64(10))\n\tassert.NoError(t, err)\n\tsorted, err := filled.Sort(ctx, functions.Col(\"id\").Asc())\n\tassert.NoError(t, err)\n\tres, err := sorted.Collect(ctx)\n\tassert.NoError(t, err)\n\trequire.Equal(t, 2, len(res))\n\tassert.Equal(t, []any{int64(1), int64(10), int64(1)}, res[0].Values())\n\tassert.Equal(t, []any{int64(10), int64(12), int64(10)}, res[1].Values())\n\n\t// specific columns\n\tfilled, err = df.FillNa(ctx, types.Int64(10), \"int\", \"int2\")\n\tassert.NoError(t, err)\n\tsorted, err = filled.Sort(ctx, functions.Col(\"id\").Asc())\n\tassert.NoError(t, err)\n\tres, err = sorted.Collect(ctx)\n\tassert.NoError(t, err)\n\trequire.Equal(t, 2, len(res))\n\tassert.Equal(t, []any{nil, int64(12), int64(10)}, res[0].Values())\n\tassert.Equal(t, []any{int64(1), int64(10), int64(1)}, res[1].Values())\n\n\t// specific columns with map\n\tfilled, err = df.FillNaWithValues(ctx, map[string]types.PrimitiveTypeLiteral{\n\t\t\"int\": types.Int64(10), \"int2\": types.Int64(20),\n\t})\n\tassert.NoError(t, err)\n\tsorted, err = filled.Sort(ctx, functions.Col(\"id\").Asc())\n\tassert.NoError(t, err)\n\tres, err = sorted.Collect(ctx)\n\tassert.NoError(t, err)\n\trequire.Equal(t, 2, len(res))\n\tassert.Equal(t, []any{nil, int64(12), int64(20)}, res[0].Values())\n\tassert.Equal(t, []any{int64(1), int64(10), int64(1)}, res[1].Values())\n}\n\nfunc TestDataFrame_ApproxQuantile(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select id, 1 as id2 from range(100)\")\n\tassert.NoError(t, err)\n\tres, err := df.ApproxQuantile(ctx, []float64{float64(0.5)}, float64(0.1), \"id\")\n\tassert.NoError(t, err)\n\tassert.Len(t, res, 1)\n\n\tdata := [][]any{\n\t\t{\"bob\", \"Developer\", 125000, 1},\n\t\t{\"mark\", \"Developer\", 108000, 2},\n\t\t{\"carl\", \"Tester\", 70000, 2},\n\t\t{\"peter\", \"Developer\", 185000, 2},\n\t\t{\"jon\", \"Tester\", 65000, 1},\n\t\t{\"roman\", \"Tester\", 82000, 2},\n\t\t{\"simon\", \"Developer\", 98000, 1},\n\t\t{\"eric\", \"Developer\", 144000, 2},\n\t\t{\"carlos\", \"Tester\", 75000, 1},\n\t\t{\"henry\", \"Developer\", 110000, 1},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"Name\", types.STRING),\n\t\ttypes.NewStructField(\"Role\", types.STRING),\n\t\ttypes.NewStructField(\"Salary\", types.LONG),\n\t\ttypes.NewStructField(\"Performance\", types.LONG),\n\t)\n\n\tdf, err = spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\tmed, err := df.ApproxQuantile(ctx, []float64{float64(0.5)}, float64(0.25), \"Salary\")\n\n\tassert.NoError(t, err)\n\tassert.Len(t, med, 1)\n\tassert.GreaterOrEqual(t, med[0][0], 75000.0)\n\n\t_, err = df.Stat().ApproxQuantile(ctx, []float64{0.5}, 0.25, \"Salary\")\n\tassert.NoError(t, err)\n}\n\nfunc TestDataFrame_DFNaFunctions(t *testing.T) {\n\tctx, spark := connect()\n\tdata := [][]any{\n\t\t{10, 80.5, \"Alice\", true},\n\t\t{5, nil, \"Bob\", nil},\n\t\t{nil, nil, \"Tom\", nil},\n\t\t{nil, nil, nil, nil},\n\t}\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"age\", types.INTEGER),\n\t\ttypes.NewStructField(\"height\", types.DOUBLE),\n\t\ttypes.NewStructField(\"name\", types.STRING),\n\t\ttypes.NewStructField(\"bool\", types.BOOLEAN),\n\t)\n\tdf, err := spark.CreateDataFrame(ctx, data, schema)\n\tassert.NoError(t, err)\n\n\tres, err := df.Na().Drop(ctx)\n\tassert.NoError(t, err)\n\trows, err := res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 1)\n\tassert.Equal(t, rows[0].At(2), \"Alice\")\n\n\tres, err = df.Na().DropAll(ctx)\n\tassert.NoError(t, err)\n\trows, err = res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 3)\n\n\t// Fill must only use long types\n\tres, err = df.Na().Fill(ctx, types.Int64(50))\n\tassert.NoError(t, err)\n\trows, err = res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 4)\n\n\tassert.Equal(t, int32(50), rows[2].At(0))\n\tassert.Equal(t, int32(50), rows[3].At(0))\n\tassert.Equal(t, float64(50), rows[2].At(1))\n\tassert.Equal(t, float64(50), rows[3].At(1))\n\n\tres, err = df.Na().Replace(ctx, []types.PrimitiveTypeLiteral{types.String(\"Alice\")}, []types.PrimitiveTypeLiteral{\n\t\ttypes.String(\"Bob\"),\n\t})\n\tassert.NoError(t, err)\n\trows, err = res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 4)\n\n\tassert.Equal(t, \"Bob\", rows[0].At(2))\n}\n\nfunc TestDataFrame_RangeIter(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\tcnt := 0\n\tfor row, err := range df.All(ctx) {\n\t\tassert.NoError(t, err)\n\t\tassert.NotNil(t, row)\n\t\tcnt++\n\t}\n\tassert.Equal(t, 10, cnt)\n\n\t// Check that errors are properly propagated\n\tdf, err = spark.Sql(ctx, \"select if(id = 5, raise_error('handle'), false) from range(10)\")\n\tassert.NoError(t, err)\n\tfor _, err := range df.All(ctx) {\n\t\t// The error is immediately thrown:\n\t\tassert.Error(t, err)\n\t}\n}\n\nfunc TestDataFrame_PrintSchema(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select * from range(10)\")\n\tassert.NoError(t, err)\n\terr = df.PrintSchema(ctx)\n\tassert.NoError(t, err)\n}\n\nfunc TestDataFrame_SchemaTreeString(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select map('a', 1) as first, array(1,2,3) as second, map('a', map('b', 2)) as third\")\n\tassert.NoError(t, err)\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tts := schema.TreeString()\n\tassert.Contains(t, ts, \"|-- first: map\")\n\tassert.Contains(t, ts, \"|-- second: array\")\n\tassert.Contains(t, ts, \"|-- third: map\")\n}\n"
  },
  {
    "path": "internal/tests/integration/functions_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage integration\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestIntegration_BuiltinFunctions(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tif err != nil {\n\t\tt.Fatal(err)\n\t}\n\n\tdf, _ := spark.Sql(ctx, \"select '[2]' as a from range(10)\")\n\tdf, _ = df.Filter(ctx, functions.JsonArrayLength(functions.Col(\"a\")).Eq(functions.IntLit(1)))\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 10, len(res))\n}\n\nfunc TestAggregationFunctions_Agg(t *testing.T) {\n\tctx, spark := connect()\n\tdf, err := spark.Sql(ctx, \"select id, 1, 2, 3 from range(100)\")\n\tassert.NoError(t, err)\n\n\tres, err := df.Agg(ctx, functions.Count(functions.Col(\"id\")))\n\tassert.NoError(t, err)\n\tcnt, err := res.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int64(1), cnt)\n\n\tres, err = df.AggWithMap(ctx, map[string]string{\"id\": \"sum\"})\n\tassert.NoError(t, err)\n\trows, err := res.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Len(t, rows, 1)\n\tassert.Equal(t, int64(4950), rows[0].At(0))\n}\n\nfunc TestIntegration_ColumnGetItem(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tif err != nil {\n\t\tt.Fatal(err)\n\t}\n\n\tdf, _ := spark.Sql(ctx, \"select sequence(1,10) as s\")\n\tdf, err = df.Select(ctx, functions.Col(\"s\").GetItem(types.Int64(2)))\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, int32(3), res[0].Values()[0])\n}\n"
  },
  {
    "path": "internal/tests/integration/helper.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage integration\n\nimport (\n\t\"context\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/arrow-go/v18/arrow/memory\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n)\n\nfunc connect() (context.Context, sql.SparkSession) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tif err != nil {\n\t\tpanic(err)\n\t}\n\treturn ctx, spark\n}\n\nfunc createArrowTable() arrow.Table {\n\tpool := memory.NewGoAllocator()\n\tschema := arrow.NewSchema(\n\t\t[]arrow.Field{\n\t\t\t{Name: \"f1-i32\", Type: arrow.PrimitiveTypes.Int32},\n\t\t\t{Name: \"f2-f64\", Type: arrow.PrimitiveTypes.Float64},\n\t\t\t{Name: \"f3-string\", Type: &arrow.StringType{}},\n\t\t},\n\t\tnil,\n\t)\n\n\tb := array.NewRecordBuilder(pool, schema)\n\tdefer b.Release()\n\n\tb.Field(0).(*array.Int32Builder).AppendValues([]int32{1, 2, 3}, nil)\n\tb.Field(1).(*array.Float64Builder).AppendValues([]float64{1.1, 2.2, 3.3}, nil)\n\tb.Field(2).(*array.StringBuilder).AppendValues([]string{\"a\", \"b\", \"c\"}, nil)\n\trec1 := b.NewRecord()\n\t// Do not release the table\n\ttbl := array.NewTableFromRecords(schema, []arrow.Record{rec1})\n\treturn tbl\n}\n"
  },
  {
    "path": "internal/tests/integration/spark_runner.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage integration\n\nimport (\n\t\"fmt\"\n\t\"io\"\n\t\"net\"\n\t\"os\"\n\t\"os/exec\"\n\t\"time\"\n\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\nfunc StartSparkConnect() (int64, error) {\n\tsparkHome := os.Getenv(\"SPARK_HOME\")\n\tif sparkHome == \"\" {\n\t\treturn -1, sparkerrors.WithString(sparkerrors.TestSetupError, \"SPARK_HOME not set\")\n\t}\n\n\tfmt.Printf(\"Starting Spark Connect Server in: %v\\n\", os.Getenv(\"SPARK_HOME\"))\n\n\tcmd := exec.Command(\"./sbin/start-connect-server.sh\", \"--conf\",\n\t\t\"spark.log.structuredLogging.enabled=false\")\n\tcmd.Dir = sparkHome\n\tbaseEnv := os.Environ()\n\tbaseEnv = append(baseEnv, \"SPARK_NO_DAEMONIZE=1\")\n\tcmd.Env = baseEnv\n\n\tstdout, _ := cmd.StdoutPipe()\n\tif err := cmd.Start(); err != nil {\n\t\treturn -1, sparkerrors.WithType(sparkerrors.TestSetupError, err)\n\t}\n\n\ttimeout := time.After(60 * time.Second)\n\ttick := time.NewTicker(1 * time.Second)\n\n\tfor {\n\t\tselect {\n\t\tcase <-timeout:\n\t\t\tout, _ := io.ReadAll(stdout)\n\t\t\tfmt.Printf(\"Output: %v\\n\", string(out))\n\t\t\treturn -1, sparkerrors.WithString(sparkerrors.TestSetupError,\n\t\t\t\t\"timeout waiting for Spark Connect to start\")\n\t\tcase <-tick.C:\n\t\t\tif cmd.ProcessState != nil && cmd.ProcessState.Exited() {\n\t\t\t\treturn -1, sparkerrors.WithString(sparkerrors.TestSetupError, \"Spark Connect exited\")\n\t\t\t}\n\t\t\tconn, err := net.Dial(\"tcp\", \"localhost:15002\")\n\t\t\tif err == nil {\n\t\t\t\tconn.Close()\n\t\t\t\treturn int64(cmd.Process.Pid), nil\n\t\t\t}\n\t\t}\n\t}\n}\n\nfunc StopSparkConnect(pid int64) error {\n\treturn nil\n}\n"
  },
  {
    "path": "internal/tests/integration/sql_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage integration\n\nimport (\n\t\"context\"\n\t\"fmt\"\n\t\"log\"\n\t\"os\"\n\t\"testing\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/column\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestIntegration_RunSQLCommand(t *testing.T) {\n\t// Run SQL command\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tif err != nil {\n\t\tt.Fatal(err)\n\t}\n\n\tdf, err := spark.Sql(ctx, \"select * from range(100)\")\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 100, len(res))\n\n\tdf, err = df.Filter(ctx, column.OfDF(df, \"id\").Lt(functions.IntLit(10)))\n\tassert.NoError(t, err)\n\tres, err = df.Collect(ctx)\n\tassert.NoErrorf(t, err, \"Must be able to collect the rows.\")\n\tassert.Equal(t, 10, len(res))\n}\n\nfunc TestIntegration_Schema(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tassert.NoError(t, err)\n\n\tdf, err := spark.Sql(ctx, \"select * from range(1)\")\n\tassert.NoError(t, err)\n\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\n\tassert.Len(t, schema.Fields, 1)\n\tassert.Equal(t, \"id\", schema.Fields[0].Name)\n\tassert.Equal(t, types.LongType{}, schema.Fields[0].DataType)\n}\n\nfunc TestIntegration_StructConversion(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(\"sc://localhost\").Build(ctx)\n\tif err != nil {\n\t\tt.Fatal(err)\n\t}\n\n\tquery := `\n\t\tselect named_struct(\n\t\t\t'a', 1,\n\t\t\t'b', 2,\n\t\t\t'c', cast(10.32 as double),\n\t\t\t'd', array(1, 2, 3, 4)\n\t\t) struct_col\n\t`\n\tdf, err := spark.Sql(ctx, query)\n\tassert.NoError(t, err)\n\tres, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(res))\n\n\tcolumnData := res[0].Values()[0]\n\tassert.NotNil(t, columnData)\n\tstructDataMap, ok := columnData.(map[string]any)\n\tassert.True(t, ok)\n\n\tassert.Contains(t, structDataMap, \"a\")\n\tassert.Contains(t, structDataMap, \"b\")\n\tassert.Contains(t, structDataMap, \"c\")\n\tassert.Contains(t, structDataMap, \"d\")\n\n\tassert.Equal(t, int32(1), structDataMap[\"a\"])\n\tassert.Equal(t, int32(2), structDataMap[\"b\"])\n\tassert.Equal(t, float64(10.32), structDataMap[\"c\"])\n\tarrayData := []any{int32(1), int32(2), int32(3), int32(4)}\n\tassert.Equal(t, arrayData, structDataMap[\"d\"])\n\n\tschema, err := df.Schema(ctx)\n\tassert.NoError(t, err)\n\tassert.Equal(t, \"struct_col\", schema.Fields[0].Name)\n}\n\nfunc TestMain(m *testing.M) {\n\tenvShouldStartService := os.Getenv(\"START_SPARK_CONNECT_SERVICE\")\n\tshouldStartService := envShouldStartService == \"\" || envShouldStartService == \"1\"\n\tpid := int64(-1)\n\tvar err error\n\n\tif shouldStartService {\n\t\tfmt.Println(\"Starting Spark Connect service...\")\n\t\tpid, err = StartSparkConnect()\n\t\tif err != nil {\n\t\t\tlog.Fatal(err)\n\t\t}\n\t}\n\tcode := m.Run()\n\tif shouldStartService {\n\t\tif err = StopSparkConnect(pid); err != nil {\n\t\t\tlog.Fatal(err)\n\t\t}\n\t}\n\tos.Exit(code)\n}\n"
  },
  {
    "path": "java/.gitignore",
    "content": "project\ntarget"
  },
  {
    "path": "java/README.md",
    "content": "# Sample Spark-Submit Wrapper\n\nThis directory provides a simple wrapper library that can be used to submit a Spark Connect Go application to a Spark Cluster. \n\n## Wrapper Library\n\nThe wrapper library expects to variable input values:\n\n1. The path to the binary file that contains the Spark Connect Go application. This path is specified via the Spark conf property `spark.golang.binary`.\n2. The actual binary has to be submitted as part of the application using the `--files` parameter to the `spark-submit` script.\n\nBuilding the libary can be done using:\n\n```bash\nsbt package\n```\n\n## Run Script\n\nThe `run.sh` script is a simple script that can be used to submit a Spark Connect Go application to a Spark Cluster. The script can be called as follows:\n\n```bash\nexport SPARK_HOME=/path/to/spark\n./run.sh ../cmd/spark-connect-example-spark-session/spark-connect-example-spark-session\n```\n\nWhen this is called from the current directory and with the Spark Connect Golang client build, it will submit the example application to the Spark Cluster.\n\nThe `run.sh` script can be modified according to your needs."
  },
  {
    "path": "java/build.sbt",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\nThisBuild / version := \"0.1.0-SNAPSHOT\"\n\nThisBuild / scalaVersion := \"2.13.16\"\n\nlazy val root = (project in file(\".\"))\n  .settings(\n    name := \"SparkConnectGoRunner\"\n  )\n\nlibraryDependencies += \"org.apache.spark\" %% \"spark-sql-api\" % \"4.0.0\"\nlibraryDependencies += \"org.apache.spark\" %% \"spark-sql\" % \"4.0.0\"\nlibraryDependencies += \"org.apache.spark\" %% \"spark-core\" % \"4.0.0\"\nlibraryDependencies += \"org.apache.spark\" %% \"spark-connect-common\" % \"4.0.0\"\nlibraryDependencies += \"org.apache.spark\" %% \"spark-connect\" % \"4.0.0\""
  },
  {
    "path": "java/run.sh",
    "content": "#!/bin/bash\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\nset -e\n\nSCALA_VERSION=2.13\nSPARK_VERSION=4.0.0\n\nif [ -z \"$SPARK_HOME\" ]; then\n  echo \"SPARK_HOME must be set to run this script.\"\n  exit 1\nfi\n\nBINARY_PATH=$1\n\nif [ -z \"$BINARY_PATH\" ]; then\n  echo \"Usage: $0 <path-to-binary>\"\n  exit 1\nfi\n\n# Check if the binary exists.\nif [ ! -f \"$BINARY_PATH\" ]; then\n  echo \"Binary not found: $BINARY_PATH, make sure the path is valid.\"\n  exit 1\nfi\n\n# Get the absolute path of the binary.\nBINARY_PATH=$(realpath $BINARY_PATH)\nBINARY_NAME=$(basename $BINARY_PATH)\n\n# Call the spark-submit script.\n$SPARK_HOME/bin/spark-submit \\\n  --files $BINARY_PATH \\\n  --conf spark.golang.binary=$BINARY_NAME \\\n  --class org.apache.spark.golang.Runner \\\n  --packages org.apache.spark:spark-connect_$SCALA_VERSION:$SPARK_VERSION \\\n  target/scala-$SCALA_VERSION/sparkconnectgorunner_$SCALA_VERSION-0.1.0-SNAPSHOT.jar"
  },
  {
    "path": "java/src/main/scala/org/apache/spark/golang/Runner.scala",
    "content": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements.  See the NOTICE file distributed with\n * this work for additional information regarding copyright ownership.\n * The ASF licenses this file to You under the Apache License, Version 2.0\n * (the \"License\"); you may not use this file except in compliance with\n * the License.  You may obtain a copy of the License at\n *\n *    http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\npackage org.apache.spark.golang\n\nimport scala.sys.process._\n\nimport org.apache.spark.{SparkContext, SparkFiles}\nimport org.apache.spark.internal.Logging\nimport org.apache.spark.sql.SparkSession\nimport org.apache.spark.sql.connect.service.SparkConnectService\n\n/**\n * This is the main entry point for the Spark Connect Go runner.\n *\n * To run any Go code on your Spark cluster using spark-submit, you can use\n * this very simple wrapper to do so. To\n */\nobject Runner extends Logging {\n  def main(args: Array[String]): Unit = {\n    // Instantiate a new Spark Context.\n    val ctx = SparkContext.getOrCreate()\n    // Start the SparkConnect service which will listen for incoming requests.\n    SparkConnectService.start(ctx)\n\n    // Create a new Spark Session to fetch the port configuration that the service\n    // listens on.\n    val spark = SparkSession.builder().getOrCreate()\n    val port = spark.conf.get(\"spark.connect.grpc.binding.port\").toInt\n\n    // Fetch the binary of the program to be executed.\n    val bin = spark.conf.get(\"spark.golang.binary\")\n\n    // Fetch the local path of the binary.\n    val path = SparkFiles.get(bin)\n    val process = Process(path, None, \"SPARK_REMOTE\" -> s\"sc://localhost:$port\")\n    process.!\n    logWarning(\"Stopping Spark Connect service\")\n    SparkConnectService.stop()\n    ctx.stop()\n  }\n}"
  },
  {
    "path": "merge_connect_go_pr.py",
    "content": "#!/usr/bin/env python3\n\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements.  See the NOTICE file distributed with\n# this work for additional information regarding copyright ownership.\n# The ASF licenses this file to You under the Apache License, Version 2.0\n# (the \"License\"); you may not use this file except in compliance with\n# the License.  You may obtain a copy of the License at\n#\n#    http://www.apache.org/licenses/LICENSE-2.0\n#\n# Unless required by applicable law or agreed to in writing, software\n# distributed under the License is distributed on an \"AS IS\" BASIS,\n# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n# See the License for the specific language governing permissions and\n# limitations under the License.\n#\n\n# Utility for creating well-formed pull request merges and pushing them to Apache\n# Spark.\n#   usage: ./merge_connect_go_pr.py    (see config env vars below)\n#\n# This utility assumes you already have a local Spark git folder and that you\n# have added remotes corresponding to both (i) the github apache Spark\n# mirror and (ii) the apache git repo.\n\nimport json\nimport os\nimport re\nimport subprocess\nimport sys\nimport traceback\nimport requests\nfrom urllib.request import urlopen\nfrom urllib.request import Request\nfrom urllib.error import HTTPError\n\ntry:\n    import jira.client\n\n    JIRA_IMPORTED = True\nexcept ImportError:\n    JIRA_IMPORTED = False\n\n# Location of your Spark git development area\nSPARK_CONNECT_GO_HOME = os.environ.get(\"SPARK_CONNECT_GO_HOME\", os.getcwd())\n# Remote name which points to the Gihub site\nPR_REMOTE_NAME = os.environ.get(\"PR_REMOTE_NAME\", \"apache-github\")\n# Remote name which points to Apache git\nPUSH_REMOTE_NAME = os.environ.get(\"PUSH_REMOTE_NAME\", \"apache\")\n# ASF JIRA username\nJIRA_USERNAME = os.environ.get(\"JIRA_USERNAME\", \"\")\n# ASF JIRA password\nJIRA_PASSWORD = os.environ.get(\"JIRA_PASSWORD\", \"\")\n# ASF JIRA Token\nJIRA_ACCESS_TOKEN = os.environ.get(\"JIRA_ACCESS_TOKEN\")\n# OAuth key used for issuing requests against the GitHub API. If this is not defined, then requests\n# will be unauthenticated. You should only need to configure this if you find yourself regularly\n# exceeding your IP's unauthenticated request rate limit. You can create an OAuth key at\n# https://github.com/settings/tokens. This script only requires the \"public_repo\" scope.\nGITHUB_OAUTH_KEY = os.environ.get(\"GITHUB_OAUTH_KEY\")\n\n\nGITHUB_BASE = \"https://github.com/apache/spark-connect-go/pull\"\nGITHUB_API_BASE = \"https://api.github.com/repos/apache/spark-connect-go\"\nJIRA_BASE = \"https://issues.apache.org/jira/browse\"\nJIRA_API_BASE = \"https://issues.apache.org/jira\"\n# Prefix added to temporary branches\nBRANCH_PREFIX = \"PR_TOOL\"\n\n\nasf_jira = None\n\ndef get_json(url):\n    try:\n        request = Request(url)\n        if GITHUB_OAUTH_KEY:\n            request.add_header(\"Authorization\", \"token %s\" % GITHUB_OAUTH_KEY)\n        return json.load(urlopen(request))\n    except HTTPError as e:\n        if \"X-RateLimit-Remaining\" in e.headers and e.headers[\"X-RateLimit-Remaining\"] == \"0\":\n            print(\n                \"Exceeded the GitHub API rate limit; see the instructions in \"\n                + \"dev/merge_connect_go_pr.py to configure an OAuth token for making authenticated \"\n                + \"GitHub requests.\"\n            )\n        else:\n            print(\"Unable to fetch URL, exiting: %s\" % url)\n        sys.exit(-1)\n\n\ndef get_pull_request(pr_num):\n    headers = {\n        \"Authorization\": f\"token {GITHUB_OAUTH_KEY}\",\n        \"Accept\": \"application/vnd.github.v3+json\",\n    }\n    response = requests.get(\n        f\"{GITHUB_API_BASE}/pulls/{pr_num}\",\n        headers=headers,\n    )\n    if response.status_code == 200:\n        return response.json()\n    else:\n        error_message = (f\"Failed to get pull request #{pr_num}. \"\n                         f\"Status code: {response.status_code}\")\n        error_message += f\"\\nResponse: {response.text}\"\n        fail(error_message)\n\n\ndef fail(msg):\n    print(msg)\n    clean_up()\n    sys.exit(-1)\n\n\ndef run_cmd(cmd):\n    print(cmd)\n    if isinstance(cmd, list):\n        return subprocess.check_output(cmd).decode(\"utf-8\")\n    else:\n        return subprocess.check_output(cmd.split(\" \")).decode(\"utf-8\")\n\n\ndef continue_maybe(prompt):\n    result = input(\"\\n%s (y/n): \" % prompt)\n    if result.lower() != \"y\":\n        fail(\"Okay, exiting\")\n\n\ndef clean_up():\n    if \"original_head\" in globals():\n        print(\"Restoring head pointer to %s\" % original_head)\n        run_cmd(\"git checkout %s\" % original_head)\n\n        branches = run_cmd(\"git branch\").replace(\" \", \"\").split(\"\\n\")\n\n        for branch in list(filter(lambda x: x.startswith(BRANCH_PREFIX), branches)):\n            print(\"Deleting local branch %s\" % branch)\n            run_cmd(\"git branch -D %s\" % branch)\n\n\n# merge the requested PR and return the merge hash\ndef merge_pr(pr_num, target_ref, title, body, pr_repo_desc):\n    pr_branch_name = \"%s_MERGE_PR_%s\" % (BRANCH_PREFIX, pr_num)\n    target_branch_name = \"%s_MERGE_PR_%s_%s\" % (BRANCH_PREFIX, pr_num, target_ref.upper())\n    run_cmd(\"git fetch %s pull/%s/head:%s\" % (PR_REMOTE_NAME, pr_num, pr_branch_name))\n    run_cmd(\"git fetch %s %s:%s\" % (PUSH_REMOTE_NAME, target_ref, target_branch_name))\n    run_cmd(\"git checkout %s\" % target_branch_name)\n\n    # Get all the data from the pull request.\n    pr = get_pull_request(pr_num)\n\n    # Check if the PR is mergeable and still open:\n    if not pr[\"mergeable\"]:\n        fail(f\"Pull request #{pr_num} is not mergeable in its current form.\")\n\n    # Check if the PR is still open.\n    if pr[\"state\"] != \"open\":\n        fail(f\"Pull request #{pr_num} is not open.\")\n\n    if pr[\"merged\"]:\n        fail(f\"Pull request #{pr_num} has already been merged.\")\n\n    if pr[\"draft\"]:\n        fail(f\"Pull request #{pr_num} is a draft.\")\n\n    # First commit author should be considered as the primary author when the rank is the same\n    commit_authors = run_cmd(\n        [\"git\", \"log\", \"%s..%s\" % (target_branch_name, pr_branch_name), \"--pretty=format:%an <%ae>\", \"--reverse\"]\n    ).split(\"\\n\")\n    distinct_authors = sorted(\n        list(dict.fromkeys(commit_authors)), key=lambda x: commit_authors.count(x), reverse=True\n    )\n    primary_author = input(\n        'Enter primary author in the format of \"name <email>\" [%s]: ' % distinct_authors[0]\n    )\n    if primary_author == \"\":\n        primary_author = distinct_authors[0]\n    else:\n        # When primary author is specified manually, de-dup it from author list and\n        # put it at the head of author list.\n        distinct_authors = list(filter(lambda x: x != primary_author, distinct_authors))\n        distinct_authors.insert(0, primary_author)\n\n    merge_message = \"\"\n    if body is not None:\n        # We remove @ symbols from the body to avoid triggering e-mails\n        # to people every time someone creates a public fork of Spark.\n        merge_message += body.replace(\"@\", \"\")\n\n    committer_name = run_cmd(\"git config --get user.name\").strip()\n    committer_email = run_cmd(\"git config --get user.email\").strip()\n\n    # The string \"Closes #%s\" string is required for GitHub to correctly close the PR\n    merge_message += \"\\n\\n\"\n    merge_message += \"Closes #%s from %s.\" % (pr_num, pr_repo_desc)\n\n    authors = \"Authored-by:\" if len(distinct_authors) == 1 else \"Lead-authored-by:\"\n    authors += \" %s\" % (distinct_authors.pop(0))\n    if len(distinct_authors) > 0:\n        authors += \"\\n\" + \"\\n\".join([\"Co-authored-by: %s\" % a for a in distinct_authors])\n    authors += \"\\n\" + \"Signed-off-by: %s <%s>\" % (committer_name, committer_email)\n\n    merge_message += \"\\n\\n\"\n    merge_message += authors\n\n    # Merge the Pull Request using the commit message and title and squash it.\n    headers = {\n        \"Authorization\": f\"token {GITHUB_OAUTH_KEY}\",\n        \"Accept\": \"application/vnd.github.v3+json\",\n    }\n\n    data = {\n        \"commit_title\": title,\n        \"commit_message\": merge_message,\n        # Must be squash, always.\n        \"merge_method\": \"squash\",\n    }\n\n    continue_maybe(\"Collected all data. Ready to merge PR?\")\n    \n    # Run the request to merge the PR.\n    response = requests.put(\n        f\"{GITHUB_API_BASE}/pulls/{pr_num}/merge\",\n        headers=headers,\n        json=data\n    )\n\n    if response.status_code == 200:\n        merge_response_json = response.json()\n        merge_commit_sha = merge_response_json.get(\"sha\")\n        print(f\"Pull request #{pr_num} merged. Sha: #{merge_commit_sha}\")\n        clean_up()\n        return merge_commit_sha\n    else:\n        error_message = f\"Failed to merge pull request #{pr_num}. Status code: {response.status_code}\"\n        error_message += f\"\\nResponse: {response.text}\"\n        clean_up()\n        fail(error_message)\n\n\ndef cherry_pick(pr_num, merge_hash, default_branch):\n    pick_ref = input(\"Enter a branch name [%s]: \" % default_branch)\n    if pick_ref == \"\":\n        pick_ref = default_branch\n\n    pick_branch_name = \"%s_PICK_PR_%s_%s\" % (BRANCH_PREFIX, pr_num, pick_ref.upper())\n\n    run_cmd(\"git fetch %s %s:%s\" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name))\n    run_cmd(\"git checkout %s\" % pick_branch_name)\n\n    try:\n        run_cmd(\"git cherry-pick -sx %s\" % merge_hash)\n    except Exception as e:\n        msg = \"Error cherry-picking: %s\\nWould you like to manually fix-up this merge?\" % e\n        continue_maybe(msg)\n        msg = \"Okay, please fix any conflicts and finish the cherry-pick. Finished?\"\n        continue_maybe(msg)\n\n    continue_maybe(\n        \"Pick complete (local ref %s). Push to %s?\" % (pick_branch_name, PUSH_REMOTE_NAME)\n    )\n\n    try:\n        run_cmd(\"git push %s %s:%s\" % (PUSH_REMOTE_NAME, pick_branch_name, pick_ref))\n    except Exception as e:\n        clean_up()\n        fail(\"Exception while pushing: %s\" % e)\n\n    pick_hash = run_cmd(\"git rev-parse %s\" % pick_branch_name)[:8]\n    clean_up()\n\n    print(\"Pull request #%s picked into %s!\" % (pr_num, pick_ref))\n    print(\"Pick hash: %s\" % pick_hash)\n    return pick_ref\n\n\ndef fix_version_from_branch(branch, versions):\n    # Note: Assumes this is a sorted (newest->oldest) list of un-released versions\n    if branch == \"master\":\n        return versions[0]\n    else:\n        branch_ver = branch.replace(\"branch-\", \"\")\n        return list(filter(lambda x: x.name.startswith(branch_ver), versions))[-1]\n\n\ndef resolve_jira_issue(merge_branches, comment, default_jira_id=\"\"):\n    global asf_jira\n\n    jira_id = input(\"Enter a JIRA id [%s]: \" % default_jira_id)\n    if jira_id == \"\":\n        jira_id = default_jira_id\n\n    try:\n        issue = asf_jira.issue(jira_id)\n    except Exception as e:\n        fail(\"ASF JIRA could not find %s\\n%s\" % (jira_id, e))\n\n    cur_status = issue.fields.status.name\n    cur_summary = issue.fields.summary\n    cur_assignee = issue.fields.assignee\n    if cur_assignee is None:\n        cur_assignee = choose_jira_assignee(issue, asf_jira)\n    # Check again, we might not have chosen an assignee\n    if cur_assignee is None:\n        cur_assignee = \"NOT ASSIGNED!!!\"\n    else:\n        cur_assignee = cur_assignee.displayName\n\n    if cur_status == \"Resolved\" or cur_status == \"Closed\":\n        fail(\"JIRA issue %s already has status '%s'\" % (jira_id, cur_status))\n    print(\"=== JIRA %s ===\" % jira_id)\n    print(\n        \"summary\\t\\t%s\\nassignee\\t%s\\nstatus\\t\\t%s\\nurl\\t\\t%s/%s\\n\"\n        % (cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)\n    )\n\n    versions = asf_jira.project_versions(\"SPARK\")\n    versions = sorted(versions, key=lambda x: x.name, reverse=True)\n    versions = list(filter(lambda x: x.raw[\"released\"] is False, versions))\n    # Consider only x.y.z versions\n    versions = list(filter(lambda x: re.match(r\"\\d+\\.\\d+\\.\\d+\", x.name), versions))\n\n    default_fix_versions = list(\n        map(lambda x: fix_version_from_branch(x, versions).name, merge_branches)\n    )\n    for v in default_fix_versions:\n        # Handles the case where we have forked a release branch but not yet made the release.\n        # In this case, if the PR is committed to the master branch and the release branch, we\n        # only consider the release branch to be the fix version. E.g. it is not valid to have\n        # both 1.1.0 and 1.0.0 as fix versions.\n        (major, minor, patch) = v.split(\".\")\n        if patch == \"0\":\n            previous = \"%s.%s.%s\" % (major, int(minor) - 1, 0)\n            if previous in default_fix_versions:\n                default_fix_versions = list(filter(lambda x: x != v, default_fix_versions))\n    default_fix_versions = \",\".join(default_fix_versions)\n\n    available_versions = set(list(map(lambda v: v.name, versions)))\n    while True:\n        try:\n            fix_versions = input(\n                \"Enter comma-separated fix version(s) [%s]: \" % default_fix_versions\n            )\n            if fix_versions == \"\":\n                fix_versions = default_fix_versions\n            fix_versions = fix_versions.replace(\" \", \"\").split(\",\")\n            if set(fix_versions).issubset(available_versions):\n                break\n            else:\n                print(\n                    \"Specified version(s) [%s] not found in the available versions, try \"\n                    \"again (or leave blank and fix manually).\" % (\", \".join(fix_versions))\n                )\n        except KeyboardInterrupt:\n            raise\n        except BaseException:\n            traceback.print_exc()\n            print(\"Error setting fix version(s), try again (or leave blank and fix manually)\")\n\n    def get_version_json(version_str):\n        return list(filter(lambda v: v.name == version_str, versions))[0].raw\n\n    jira_fix_versions = list(map(lambda v: get_version_json(v), fix_versions))\n\n    resolve = list(filter(lambda a: a[\"name\"] == \"Resolve Issue\", asf_jira.transitions(jira_id)))[0]\n    resolution = list(filter(lambda r: r.raw[\"name\"] == \"Fixed\", asf_jira.resolutions()))[0]\n    asf_jira.transition_issue(\n        jira_id,\n        resolve[\"id\"],\n        fixVersions=jira_fix_versions,\n        comment=comment,\n        resolution={\"id\": resolution.raw[\"id\"]},\n    )\n\n    print(\"Successfully resolved %s with fixVersions=%s!\" % (jira_id, fix_versions))\n\n\ndef choose_jira_assignee(issue, asf_jira):\n    \"\"\"\n    Prompt the user to choose who to assign the issue to in jira, given a list of candidates,\n    including the original reporter and all commentors\n    \"\"\"\n    while True:\n        try:\n            reporter = issue.fields.reporter\n            commentors = list(map(lambda x: x.author, issue.fields.comment.comments))\n            candidates = set(commentors)\n            candidates.add(reporter)\n            candidates = list(candidates)\n            print(\"JIRA is unassigned, choose assignee\")\n            for idx, author in enumerate(candidates):\n                if author.key == \"apachespark\":\n                    continue\n                annotations = [\"Reporter\"] if author == reporter else []\n                if author in commentors:\n                    annotations.append(\"Commentor\")\n                print(\"[%d] %s (%s)\" % (idx, author.displayName, \",\".join(annotations)))\n            raw_assignee = input(\n                \"Enter number of user, or userid, to assign to (blank to leave unassigned):\"\n            )\n            if raw_assignee == \"\":\n                return None\n            else:\n                try:\n                    id = int(raw_assignee)\n                    assignee = candidates[id]\n                except BaseException:\n                    # assume it's a user id, and try to assign (might fail, we just prompt again)\n                    assignee = asf_jira.user(raw_assignee)\n                asf_jira.assign_issue(issue.key, assignee.name)\n                return assignee\n        except KeyboardInterrupt:\n            raise\n        except BaseException:\n            traceback.print_exc()\n            print(\"Error assigning JIRA, try again (or leave blank and fix manually)\")\n\n\ndef resolve_jira_issues(title, merge_branches, comment):\n    jira_ids = re.findall(\"SPARK-[0-9]{4,5}\", title)\n\n    if len(jira_ids) == 0:\n        resolve_jira_issue(merge_branches, comment)\n    for jira_id in jira_ids:\n        resolve_jira_issue(merge_branches, comment, jira_id)\n\n\ndef standardize_jira_ref(text):\n    \"\"\"\n    Standardize the [SPARK-XXXXX] [MODULE] prefix\n    Converts \"[SPARK-XXX][mllib] Issue\", \"[MLLib] SPARK-XXX. Issue\" or \"SPARK XXX [MLLIB]: Issue\" to\n    \"[SPARK-XXX][MLLIB] Issue\"\n\n    >>> standardize_jira_ref(\n    ...     \"[SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful\")\n    '[SPARK-5821][SQL] ParquetRelation2 CTAS should check if delete is successful'\n    >>> standardize_jira_ref(\n    ...     \"[SPARK-4123][Project Infra][WIP]: Show new dependencies added in pull requests\")\n    '[SPARK-4123][PROJECT INFRA][WIP] Show new dependencies added in pull requests'\n    >>> standardize_jira_ref(\"[MLlib] Spark  5954: Top by key\")\n    '[SPARK-5954][MLLIB] Top by key'\n    >>> standardize_jira_ref(\"[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl\")\n    '[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl'\n    >>> standardize_jira_ref(\n    ...     \"SPARK-1094 Support MiMa for reporting binary compatibility across versions.\")\n    '[SPARK-1094] Support MiMa for reporting binary compatibility across versions.'\n    >>> standardize_jira_ref(\"[WIP]  [SPARK-1146] Vagrant support for Spark\")\n    '[SPARK-1146][WIP] Vagrant support for Spark'\n    >>> standardize_jira_ref(\n    ...     \"SPARK-1032. If Yarn app fails before registering, app master stays aroun...\")\n    '[SPARK-1032] If Yarn app fails before registering, app master stays aroun...'\n    >>> standardize_jira_ref(\n    ...     \"[SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.\")\n    '[SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.'\n    >>> standardize_jira_ref(\"Additional information for users building from source code\")\n    'Additional information for users building from source code'\n    \"\"\"\n    jira_refs = []\n    components = []\n\n    # If the string is compliant, no need to process any further\n    if re.search(r\"^\\[SPARK-[0-9]{3,6}\\](\\[[A-Z0-9_\\s,]+\\] )+\\S+\", text):\n        return text\n\n    # Extract JIRA ref(s):\n    pattern = re.compile(r\"(SPARK[-\\s]*[0-9]{3,6})+\", re.IGNORECASE)\n    for ref in pattern.findall(text):\n        # Add brackets, replace spaces with a dash, & convert to uppercase\n        jira_refs.append(\"[\" + re.sub(r\"\\s+\", \"-\", ref.upper()) + \"]\")\n        text = text.replace(ref, \"\")\n\n    # Extract spark component(s):\n    # Look for alphanumeric chars, spaces, dashes, periods, and/or commas\n    pattern = re.compile(r\"(\\[[\\w\\s,.-]+\\])\", re.IGNORECASE)\n    for component in pattern.findall(text):\n        components.append(component.upper())\n        text = text.replace(component, \"\")\n\n    # Cleanup any remaining symbols:\n    pattern = re.compile(r\"^\\W+(.*)\", re.IGNORECASE)\n    if pattern.search(text) is not None:\n        text = pattern.search(text).groups()[0]\n\n    # Assemble full text (JIRA ref(s), module(s), remaining text)\n    clean_text = \"\".join(jira_refs).strip() + \"\".join(components).strip() + \" \" + text.strip()\n\n    # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were\n    # included\n    clean_text = re.sub(r\"\\s+\", \" \", clean_text.strip())\n\n    return clean_text\n\n\ndef get_current_ref():\n    ref = run_cmd(\"git rev-parse --abbrev-ref HEAD\").strip()\n    if ref == \"HEAD\":\n        # The current ref is a detached HEAD, so grab its SHA.\n        return run_cmd(\"git rev-parse HEAD\").strip()\n    else:\n        return ref\n\n\ndef initialize_jira():\n    global asf_jira\n    jira_server = {\"server\": JIRA_API_BASE}\n\n    if not JIRA_IMPORTED:\n        print_error(\"ERROR finding jira library. Run 'pip3 install jira' to install.\")\n        continue_maybe(\"Continue without jira?\")\n    elif JIRA_ACCESS_TOKEN:\n        client = jira.client.JIRA(jira_server, token_auth=JIRA_ACCESS_TOKEN)\n        try:\n            # Eagerly check if the token is valid to align with the behavior of username/password\n            # authn\n            client.current_user()\n            asf_jira = client\n        except Exception as e:\n            if e.__class__.__name__ == \"JIRAError\" and getattr(e, \"status_code\", None) == 401:\n                msg = (\n                    \"ASF JIRA could not authenticate with the invalid or expired token '%s'\"\n                    % JIRA_ACCESS_TOKEN\n                )\n                fail(msg)\n            else:\n                raise e\n    elif JIRA_USERNAME and JIRA_PASSWORD:\n        print(\"You can use JIRA_ACCESS_TOKEN instead of JIRA_USERNAME/JIRA_PASSWORD.\")\n        print(\"Visit https://issues.apache.org/jira/secure/ViewProfile.jspa \")\n        print(\"and click 'Personal Access Tokens' menu to manage your own tokens.\")\n        asf_jira = jira.client.JIRA(jira_server, basic_auth=(JIRA_USERNAME, JIRA_PASSWORD))\n    else:\n        print(\"Neither JIRA_ACCESS_TOKEN nor JIRA_USERNAME/JIRA_PASSWORD are set.\")\n        continue_maybe(\"Continue without jira?\")\n\n\ndef main():\n    global original_head\n    global asf_jira\n\n    initialize_jira()\n\n    os.chdir(SPARK_CONNECT_GO_HOME)\n    original_head = get_current_ref()\n\n    branches = get_json(\"%s/branches\" % GITHUB_API_BASE)\n    branch_names = list(filter(lambda x: x.startswith(\"branch-\"), [x[\"name\"] for x in branches]))\n    # Assumes branch names can be sorted lexicographically\n    if len(branch_names) == 0:\n        # Remove this when we have a branch. It fails now because we don't have branch-*.\n        latest_branch = \"master\"\n    else:\n        latest_branch = sorted(branch_names, reverse=True)[0]\n\n    pr_num = input(\"Which pull request would you like to merge? (e.g. 34): \")\n    pr = get_json(\"%s/pulls/%s\" % (GITHUB_API_BASE, pr_num))\n    pr_events = get_json(\"%s/issues/%s/events\" % (GITHUB_API_BASE, pr_num))\n\n    url = pr[\"url\"]\n\n    # Warn if the PR is WIP\n    if \"[WIP]\" in pr[\"title\"]:\n        msg = \"The PR title has `[WIP]`:\\n%s\\nContinue?\" % pr[\"title\"]\n        continue_maybe(msg)\n\n    # Decide whether to use the modified title or not\n    modified_title = standardize_jira_ref(pr[\"title\"]).rstrip(\".\")\n    if modified_title != pr[\"title\"]:\n        print(\"I've re-written the title as follows to match the standard format:\")\n        print(\"Original: %s\" % pr[\"title\"])\n        print(\"Modified: %s\" % modified_title)\n        result = input(\"Would you like to use the modified title? (y/n): \")\n        if result.lower() == \"y\":\n            title = modified_title\n            print(\"Using modified title:\")\n        else:\n            title = pr[\"title\"]\n            print(\"Using original title:\")\n        print(title)\n    else:\n        title = pr[\"title\"]\n\n    body = pr[\"body\"]\n    if body is None:\n        body = \"\"\n    modified_body = re.sub(re.compile(r\"<!--[^>]*-->\\n?\", re.DOTALL), \"\", body).lstrip()\n    if modified_body != body:\n        print(\"=\" * 80)\n        print(modified_body)\n        print(\"=\" * 80)\n        print(\"I've removed the comments from PR template like the above:\")\n        result = input(\"Would you like to use the modified body? (y/n): \")\n        if result.lower() == \"y\":\n            body = modified_body\n            print(\"Using modified body:\")\n        else:\n            print(\"Using original body:\")\n        print(\"=\" * 80)\n        print(body)\n        print(\"=\" * 80)\n    target_ref = pr[\"base\"][\"ref\"]\n    user_login = pr[\"user\"][\"login\"]\n    base_ref = pr[\"head\"][\"ref\"]\n    pr_repo_desc = \"%s/%s\" % (user_login, base_ref)\n\n    # Merged pull requests don't appear as merged in the GitHub API;\n    # Instead, they're closed by asfgit.\n    merge_commits = [\n        e for e in pr_events if e[\"actor\"][\"login\"] == \"asfgit\" and e[\"event\"] == \"closed\"\n    ]\n\n    if merge_commits:\n        merge_hash = merge_commits[0][\"commit_id\"]\n        message = get_json(\"%s/commits/%s\" % (GITHUB_API_BASE, merge_hash))[\"commit\"][\"message\"]\n\n        print(\"Pull request %s has already been merged, assuming you want to backport\" % pr_num)\n        commit_is_downloaded = (\n            run_cmd([\"git\", \"rev-parse\", \"--quiet\", \"--verify\", \"%s^{commit}\" % merge_hash]).strip()\n            != \"\"\n        )\n        if not commit_is_downloaded:\n            fail(\"Couldn't find any merge commit for #%s, you may need to update HEAD.\" % pr_num)\n\n        print(\"Found commit %s:\\n%s\" % (merge_hash, message))\n        cherry_pick(pr_num, merge_hash, latest_branch)\n        sys.exit(0)\n\n    if not bool(pr[\"mergeable\"]):\n        msg = (\n            \"Pull request %s is not mergeable in its current form.\\n\" % pr_num\n            + \"Continue? (experts only!)\"\n        )\n        continue_maybe(msg)\n\n    print(\"\\n=== Pull Request #%s ===\" % pr_num)\n    print(\"title\\t%s\\nsource\\t%s\\ntarget\\t%s\\nurl\\t%s\" % (title, pr_repo_desc, target_ref, url))\n    continue_maybe(\"Proceed with merging pull request #%s?\" % pr_num)\n\n    merged_refs = [target_ref]\n\n    merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc)\n\n    pick_prompt = \"Would you like to pick %s into another branch?\" % merge_hash\n    while input(\"\\n%s (y/n): \" % pick_prompt).lower() == \"y\":\n        merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)]\n\n    if JIRA_IMPORTED:\n        if asf_jira is not None:\n            continue_maybe(\"Would you like to update an associated JIRA?\")\n            jira_comment = \"Issue resolved by pull request %s\\n[%s/%s]\" % (\n                pr_num,\n                GITHUB_BASE,\n                pr_num,\n            )\n            resolve_jira_issues(title, merged_refs, jira_comment)\n        else:\n            print(\"JIRA_USERNAME and JIRA_PASSWORD not set\")\n            print(\"Exiting without trying to close the associated JIRA.\")\n    else:\n        print(\"Could not find jira-python library. Run 'sudo pip3 install jira' to install.\")\n        print(\"Exiting without trying to close the associated JIRA.\")\n\n\nif __name__ == \"__main__\":\n    import doctest\n\n    (failure_count, test_count) = doctest.testmod()\n    if failure_count:\n        sys.exit(-1)\n    try:\n        main()\n    except BaseException:\n        clean_up()\n        raise\n"
  },
  {
    "path": "quick-start.md",
    "content": "# Quick Start Guide to Write Spark Connect Client Application in Go\n\n## Add Reference to `spark-connect-go` Library\n\nIn your Go project `go.mod` file, add `spark-connect-go` library:\n```\nrequire (\n\tgithub.com/apache/spark-connect-go master\n)\n```\n\nIn your Go project, run `go mod tidy` to download the library on your local machine.\n\n## Write Spark Connect Client Application\n\nCreate `main.go` file with following code:\n```go\npackage main\n\nimport (\n\t\"context\"\n\t\"flag\"\n\t\"fmt\"\n\t\"log\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql\"\n)\n\nvar (\n\tremote = flag.String(\"remote\", \"sc://localhost:15002\",\n\t\t\"the remote address of Spark Connect server to connect to\")\n\n\tfiledir = flag.String(\"filedir\", \"/tmp\",\n\t\t\"the directory to save the files\")\n)\n\nfunc main() {\n\tflag.Parse()\n\tctx := context.Background()\n\tspark, err := sql.NewSessionBuilder().Remote(*remote).Build(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\tdefer spark.Stop()\n\n\tdf, err := spark.Sql(ctx, \"select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from sql: select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tschema, err := df.Schema(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tfor _, f := range schema.Fields {\n\t\tlog.Printf(\"Field in dataframe schema: %s - %s\", f.Name, f.DataType.TypeName())\n\t}\n\n\trows, err := df.Collect(ctx)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tschema, err = rows[0].Schema()\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tfor _, f := range schema.Fields {\n\t\tlog.Printf(\"Field in row: %s - %s\", f.Name, f.DataType.TypeName())\n\t}\n\n\tfor _, row := range rows {\n\t\tlog.Printf(\"Row: %v\", row)\n\t}\n\n\terr = df.Writer().Mode(\"overwrite\").\n\t\tFormat(\"parquet\").\n\t\tSave(ctx, fmt.Sprintf(\"file://%s/spark-connect-write-example-output.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Read().Format(\"parquet\").\n\t\tLoad(fmt.Sprintf(\"file://%s/spark-connect-write-example-output.parquet\", *filedir))\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from reading parquet\")\n\terr = df.Show(ctx, 100, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\terr = df.CreateTempView(ctx, \"view1\", true, false)\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tdf, err = spark.Sql(ctx, \"select count, word from view1 order by count\")\n\tif err != nil {\n\t\tlog.Fatalf(\"Failed: %s\", err)\n\t}\n\n\tlog.Printf(\"DataFrame from sql: select count, word from view1 order by count\")\n\tdf.Show(ctx, 100, false)\n}\n```\n\n## Start Spark Connect Server (Driver)\n\nDownload a Spark distribution (4.0.0+), unzip the folder, run command:\n```\nsbin/start-connect-server.sh\n```\n\n## Run Spark Connect Client Application\n```\ngo run main.go --filedir YOUR_TMP_DIR\n```\n\nYou will see the client application connects to the Spark Connect server and prints out the output from your application.\n"
  },
  {
    "path": "spark/client/base/base.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage base\n\nimport (\n\t\"context\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n)\n\ntype SparkConnectRPCClient generated.SparkConnectServiceClient\n\n// SparkConnectClient is the interface for executing a plan in Spark.\n//\n// This interface does not deal with the public Spark API abstractions but roughly deals on the\n// RPC API level and the necessary translation of Arrow to Row objects.\ntype SparkConnectClient interface {\n\tExecutePlan(ctx context.Context, plan *generated.Plan) (ExecuteResponseStream, error)\n\tExecuteCommand(ctx context.Context, plan *generated.Plan) (arrow.Table, *types.StructType, map[string]any, error)\n\tAnalyzePlan(ctx context.Context, plan *generated.Plan) (*generated.AnalyzePlanResponse, error)\n\tExplain(ctx context.Context, plan *generated.Plan, explainMode utils.ExplainMode) (*generated.AnalyzePlanResponse, error)\n\tPersist(ctx context.Context, plan *generated.Plan, storageLevel utils.StorageLevel) error\n\tUnpersist(ctx context.Context, plan *generated.Plan) error\n\tGetStorageLevel(ctx context.Context, plan *generated.Plan) (*utils.StorageLevel, error)\n\tSparkVersion(ctx context.Context) (string, error)\n\tDDLParse(ctx context.Context, sql string) (*types.StructType, error)\n\tSameSemantics(ctx context.Context, plan1 *generated.Plan, plan2 *generated.Plan) (bool, error)\n\tSemanticHash(ctx context.Context, plan *generated.Plan) (int32, error)\n\tConfig(ctx context.Context, configRequest *generated.ConfigRequest_Operation) (*generated.ConfigResponse, error)\n}\n\ntype ExecuteResponseStream interface {\n\tToTable() (*types.StructType, arrow.Table, error)\n\tProperties() map[string]any\n}\n"
  },
  {
    "path": "spark/client/channel/channel.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage channel\n\nimport (\n\t\"context\"\n\t\"crypto/tls\"\n\t\"crypto/x509\"\n\t\"errors\"\n\t\"fmt\"\n\t\"net\"\n\t\"net/url\"\n\t\"os\"\n\t\"runtime\"\n\t\"strconv\"\n\t\"strings\"\n\n\t\"github.com/apache/spark-connect-go/spark\"\n\n\t\"github.com/google/uuid\"\n\n\t\"google.golang.org/grpc/credentials/insecure\"\n\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"golang.org/x/oauth2\"\n\t\"google.golang.org/grpc\"\n\t\"google.golang.org/grpc/credentials\"\n\t\"google.golang.org/grpc/credentials/oauth\"\n)\n\n// Builder is the interface that is used to implement different patterns that\n// create the GRPC connection.\n//\n// This allows other consumers to plugin custom authentication and authorization\n// handlers without having to extend directly the Spark Connect code.\ntype Builder interface {\n\t// Build creates the grpc.ClientConn according to the configuration of the builder.\n\t// Implementations are free to provide additional paramters in their implementation\n\t// and simply must satisfy this minimal set of requirements.\n\tBuild(ctx context.Context) (*grpc.ClientConn, error)\n\t// User identifies the username passed as part of the Spark Connect requests.\n\tUser() string\n\t// Headers refers to the request metadata that is passed for every request from the\n\t// client to the server.\n\tHeaders() map[string]string\n\t// SessionId identifies the client side session identifier. This value must be a UUID formatted\n\t// as a string.\n\tSessionId() string\n\t// UserAgent identifies the user agent string that is passed as part of the request. It contains\n\t// information about the operating system, Go version etc.\n\tUserAgent() string\n}\n\n// BaseBuilder is used to parse the different parameters of the connection\n// string according to the specification documented here:\n//\n//\thttps://github.com/apache/spark/blob/master/connector/connect/docs/client-connection-string.md\ntype BaseBuilder struct {\n\thost      string\n\tport      int\n\ttoken     string\n\tuser      string\n\theaders   map[string]string\n\tsessionId string\n\tuserAgent string\n}\n\nfunc (cb *BaseBuilder) Host() string {\n\treturn cb.host\n}\n\nfunc (cb *BaseBuilder) Port() int {\n\treturn cb.port\n}\n\nfunc (cb *BaseBuilder) Token() string {\n\treturn cb.token\n}\n\nfunc (cb *BaseBuilder) User() string {\n\treturn cb.user\n}\n\nfunc (cb *BaseBuilder) Headers() map[string]string {\n\treturn cb.headers\n}\n\nfunc (cb *BaseBuilder) SessionId() string {\n\treturn cb.sessionId\n}\n\nfunc (cb *BaseBuilder) UserAgent() string {\n\treturn cb.userAgent\n}\n\n// Build finalizes the creation of the gprc.ClientConn by creating a GRPC channel\n// with the necessary options extracted from the connection string. For\n// TLS connections, this function will load the system certificates.\nfunc (cb *BaseBuilder) Build(ctx context.Context) (*grpc.ClientConn, error) {\n\tvar opts []grpc.DialOption\n\n\topts = append(opts, grpc.WithAuthority(cb.host))\n\tif cb.token == \"\" {\n\t\topts = append(opts, grpc.WithTransportCredentials(insecure.NewCredentials()))\n\t} else {\n\t\t// Note: On the Windows platform, use of x509.SystemCertPool() requires\n\t\t// go version 1.18 or higher.\n\t\tsystemRoots, err := x509.SystemCertPool()\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tcred := credentials.NewTLS(&tls.Config{\n\t\t\tRootCAs: systemRoots,\n\t\t})\n\t\topts = append(opts, grpc.WithTransportCredentials(cred))\n\t\tts := oauth2.StaticTokenSource(&oauth2.Token{\n\t\t\tAccessToken: cb.token,\n\t\t\tTokenType:   \"bearer\",\n\t\t})\n\t\topts = append(opts, grpc.WithPerRPCCredentials(oauth.TokenSource{TokenSource: ts}))\n\t}\n\n\tremote := fmt.Sprintf(\"%v:%v\", cb.host, cb.port)\n\tconn, err := grpc.NewClient(remote, opts...)\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to connect to remote %s: %w\",\n\t\t\tremote, err), sparkerrors.ConnectionError)\n\t}\n\treturn conn, nil\n}\n\n// NewBuilder creates a new instance of the BaseBuilder. This constructor effectively\n// parses the connection string and extracts the relevant parameters directly.\n//\n// The following parameters to the connection string are reserved: user_id, session_id, use_ssl,\n// and token. These parameters are not allowed to be injected as headers.\nfunc NewBuilder(connection string) (*BaseBuilder, error) {\n\tu, err := url.Parse(connection)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\tif u.Hostname() == \"\" {\n\t\treturn nil, sparkerrors.WithType(errors.New(\"URL must contain a hostname\"), sparkerrors.InvalidInputError)\n\t}\n\n\tif u.Scheme != \"sc\" {\n\t\treturn nil, sparkerrors.WithType(errors.New(\"URL schema must be set to `sc`\"), sparkerrors.InvalidInputError)\n\t}\n\n\tport := 15002\n\thost := u.Host\n\t// Check if the host part of the URL contains a port and extract.\n\tif strings.Contains(u.Host, \":\") {\n\t\t// We can ignore the error here already since the url parsing\n\t\t// raises the error about invalid port.\n\t\thostStr, portStr, _ := net.SplitHostPort(u.Host)\n\t\thost = hostStr\n\t\tif len(portStr) != 0 {\n\t\t\tport, err = strconv.Atoi(portStr)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, err\n\t\t\t}\n\t\t}\n\t}\n\n\t// Validate that the URL path is empty or follows the right format.\n\tif u.Path != \"\" && !strings.HasPrefix(u.Path, \"/;\") {\n\t\treturn nil, sparkerrors.WithType(\n\t\t\tfmt.Errorf(\"the URL path (%v) must be empty or have a proper parameter syntax\", u.Path),\n\t\t\tsparkerrors.InvalidInputError)\n\t}\n\n\tcb := &BaseBuilder{\n\t\thost:      host,\n\t\tport:      port,\n\t\theaders:   map[string]string{},\n\t\tsessionId: uuid.NewString(),\n\t\tuserAgent: \"\",\n\t}\n\n\telements := strings.Split(u.Path, \";\")\n\tfor _, e := range elements {\n\t\tprops := strings.Split(e, \"=\")\n\t\tif len(props) == 2 {\n\t\t\tswitch props[0] {\n\t\t\tcase \"token\":\n\t\t\t\tcb.token = props[1]\n\t\t\tcase \"user_id\":\n\t\t\t\tcb.user = props[1]\n\t\t\tcase \"session_id\":\n\t\t\t\tcb.sessionId = props[1]\n\t\t\tcase \"user_agent\":\n\t\t\t\tcb.userAgent = props[1]\n\t\t\tdefault:\n\t\t\t\tcb.headers[props[0]] = props[1]\n\t\t\t}\n\t\t}\n\t}\n\n\t// Set default user ID if not set.\n\tif cb.user == \"\" {\n\t\tcb.user = os.Getenv(\"USER\")\n\t\tif cb.user == \"\" {\n\t\t\tcb.user = \"na\"\n\t\t}\n\t}\n\n\t// Update the user agent if it is not set or set to a custom value.\n\tval := os.Getenv(\"SPARK_CONNECT_USER_AGENT\")\n\tif cb.userAgent == \"\" && val != \"\" {\n\t\tcb.userAgent = os.Getenv(\"SPARK_CONNECT_USER_AGENT\")\n\t} else if cb.userAgent == \"\" {\n\t\tcb.userAgent = \"_SPARK_CONNECT_GO\"\n\t}\n\n\t// In addition, to the specified user agent, we need to append information about the\n\t// host encoded as user agent components.\n\tcb.userAgent = fmt.Sprintf(\"%s spark/%s os/%s go/%s\", cb.userAgent, spark.Version(), runtime.GOOS, runtime.Version())\n\n\treturn cb, nil\n}\n"
  },
  {
    "path": "spark/client/channel/channel_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage channel_test\n\nimport (\n\t\"context\"\n\t\"strings\"\n\t\"testing\"\n\n\t\"github.com/google/uuid\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/channel\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nconst goodChannelURL = \"sc://host:15002/;user_id=a;token=b;x-other-header=c\"\n\nfunc TestBasicChannelBuilder(t *testing.T) {\n\tcb, _ := channel.NewBuilder(goodChannelURL)\n\tif cb == nil {\n\t\tt.Error(\"ChannelBuilder must not be null\")\n\t}\n}\n\nfunc TestBasicChannelParsing(t *testing.T) {\n\t_, err := channel.NewBuilder(\"abc://asdada:1333\")\n\tassert.False(t, strings.Contains(err.Error(), \"scheme\"),\n\t\t\"Channel build should fail with wrong scheme\")\n\n\t_, err = channel.NewBuilder(\"sc://:1333\")\n\tassert.False(t, strings.Contains(err.Error(), \"scheme\"),\n\t\t\"Should not have an error for a proper URL\")\n\n\tcb, err := channel.NewBuilder(\"sc://empty\")\n\tassert.Nilf(t, err, \"Valid path should not fail: %v\", err)\n\tassert.Equalf(t, 15002, cb.Port(), \"Default port must be set, but got %v\", cb.Port)\n\n\t_, err = channel.NewBuilder(\"sc://empty:port\")\n\tassert.NotNilf(t, err, \"Port must be a valid integer %v\", err)\n\n\t_, err = channel.NewBuilder(\"sc://empty:9999999999999\")\n\tassert.Nilf(t, err, \"Port must be a valid number %v\", err)\n\n\t_, err = channel.NewBuilder(\"sc://abcd/this\")\n\tassert.True(t, strings.Contains(err.Error(), \"URL path\"), \"URL path elements are not allowed\")\n\tassert.ErrorIs(t, err, sparkerrors.InvalidInputError)\n\n\tcb, err = channel.NewBuilder(goodChannelURL)\n\tassert.Nilf(t, err, \"Should not have an error for a proper URL\")\n\tassert.Equal(t, \"host\", cb.Host())\n\tassert.Equal(t, 15002, cb.Port())\n\tassert.Len(t, cb.Headers(), 1)\n\tassert.Equal(t, \"c\", cb.Headers()[\"x-other-header\"])\n\tassert.Equal(t, \"a\", cb.User())\n\tassert.Equal(t, \"b\", cb.Token())\n\n\tcb, err = channel.NewBuilder(\"sc://localhost:443/;token=token;user_id=user_id;cluster_id=a;session_id=session\")\n\tassert.NoError(t, err)\n\tassert.Equal(t, 443, cb.Port())\n\tassert.Equal(t, \"localhost\", cb.Host())\n\tassert.Equal(t, \"token\", cb.Token())\n\tassert.Equal(t, \"user_id\", cb.User())\n\tassert.Equal(t, \"session\", cb.SessionId())\n}\n\nfunc TestChannelBuildConnect(t *testing.T) {\n\tctx := context.Background()\n\tcb, err := channel.NewBuilder(\"sc://localhost\")\n\tassert.NoError(t, err)\n\tid := cb.SessionId()\n\t_, err = uuid.Parse(id)\n\tassert.NoError(t, err)\n\tassert.NoError(t, err, \"Should not have an error for a proper URL.\")\n\tconn, err := cb.Build(ctx)\n\tassert.Nil(t, err, \"no error for proper connection\")\n\tassert.NotNil(t, conn)\n\n\tcb, err = channel.NewBuilder(\"sc://localhost:443/;token=abcd;user_id=a\")\n\tassert.Nil(t, err, \"Should not have an error for a proper URL.\")\n\tconn, err = cb.Build(ctx)\n\tassert.Nil(t, err, \"no error for proper connection\")\n\tassert.NotNil(t, conn)\n}\n\nfunc TestChannelBulder_UserAgent(t *testing.T) {\n\tcb, err := channel.NewBuilder(\"sc://localhost\")\n\tassert.NoError(t, err)\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"_SPARK_CONNECT_GO\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"go/\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"spark/\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"os/\"))\n\n\tcb, err = channel.NewBuilder(\"sc://localhost/;user_agent=custom\")\n\tassert.NoError(t, err)\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"custom\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"go/\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"spark/\"))\n\tassert.True(t, strings.Contains(cb.UserAgent(), \"os/\"))\n}\n"
  },
  {
    "path": "spark/client/channel/compat.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage channel\n\n// ChannelBuilder re-exports BaseBuilder as its previous name for compatibility.\n//\n// Deprecated: use BaseBuilder instead.\ntype ChannelBuilder = BaseBuilder\n"
  },
  {
    "path": "spark/client/client.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage client\n\nimport (\n\t\"context\"\n\t\"errors\"\n\t\"fmt\"\n\t\"io\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n\n\t\"google.golang.org/grpc\"\n\t\"google.golang.org/grpc/metadata\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/base\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/options\"\n\n\t\"github.com/google/uuid\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\ntype sparkConnectClientImpl struct {\n\tclient    base.SparkConnectRPCClient\n\tmetadata  metadata.MD\n\tsessionId string\n\topts      options.SparkClientOptions\n}\n\nfunc (s *sparkConnectClientImpl) newExecutePlanRequest(plan *proto.Plan) *proto.ExecutePlanRequest {\n\t// Every new executin needs to get a new operation ID.\n\toperationId := uuid.NewString()\n\treturn &proto.ExecutePlanRequest{\n\t\tSessionId: s.sessionId,\n\t\tPlan:      plan,\n\t\tUserContext: &proto.UserContext{\n\t\t\tUserId: s.opts.UserId,\n\t\t},\n\t\tClientType: &s.opts.UserAgent,\n\t\t// Operation ID is needed for being able to reattach.\n\t\tOperationId: &operationId,\n\t\tRequestOptions: []*proto.ExecutePlanRequest_RequestOption{\n\t\t\t{\n\t\t\t\tRequestOption: &proto.ExecutePlanRequest_RequestOption_ReattachOptions{\n\t\t\t\t\tReattachOptions: &proto.ReattachOptions{\n\t\t\t\t\t\tReattachable: s.opts.ReattachExecution,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n}\n\nfunc (s *sparkConnectClientImpl) ExecuteCommand(ctx context.Context, plan *proto.Plan) (arrow.Table, *types.StructType, map[string]any, error) {\n\trequest := s.newExecutePlanRequest(plan)\n\n\t// Check that the supplied plan is actually a command.\n\tif plan.GetCommand() == nil {\n\t\treturn nil, nil, nil, sparkerrors.WithType(\n\t\t\tfmt.Errorf(\"the supplied plan does not contain a command\"), sparkerrors.ExecutionError)\n\t}\n\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\tc, err := s.client.ExecutePlan(ctx, request)\n\tif err != nil {\n\t\treturn nil, nil, nil, sparkerrors.WithType(\n\t\t\tfmt.Errorf(\"failed to call ExecutePlan in session %s: %w\", s.sessionId, err), sparkerrors.ExecutionError)\n\t}\n\trespHandler := NewExecuteResponseStream(c, s.sessionId, *request.OperationId, s.opts)\n\tschema, table, err := respHandler.ToTable()\n\tif err != nil {\n\t\treturn nil, nil, nil, err\n\t}\n\treturn table, schema, respHandler.Properties(), nil\n}\n\nfunc (s *sparkConnectClientImpl) ExecutePlan(ctx context.Context, plan *proto.Plan) (base.ExecuteResponseStream, error) {\n\trequest := s.newExecutePlanRequest(plan)\n\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\tc, err := s.client.ExecutePlan(ctx, request)\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\"failed to call ExecutePlan in session %s: %w\", s.sessionId, err), sparkerrors.ExecutionError)\n\t}\n\treturn NewExecuteResponseStream(c, s.sessionId, *request.OperationId, s.opts), nil\n}\n\n// Creates a new AnalyzePlanRequest with the necessary metadata.\nfunc (s *sparkConnectClientImpl) newAnalyzePlanStub() proto.AnalyzePlanRequest {\n\treturn proto.AnalyzePlanRequest{\n\t\tSessionId: s.sessionId,\n\t\tUserContext: &proto.UserContext{\n\t\t\tUserId: s.opts.UserId,\n\t\t},\n\t\tClientType: &s.opts.UserAgent,\n\t}\n}\n\nfunc (s *sparkConnectClientImpl) AnalyzePlan(ctx context.Context, plan *proto.Plan) (*proto.AnalyzePlanResponse, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_Schema_{\n\t\tSchema: &proto.AnalyzePlanRequest_Schema{\n\t\t\tPlan: plan,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn nil, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn response, nil\n}\n\nfunc (s *sparkConnectClientImpl) Explain(ctx context.Context, plan *proto.Plan,\n\texplainMode utils.ExplainMode,\n) (*proto.AnalyzePlanResponse, error) {\n\tvar mode proto.AnalyzePlanRequest_Explain_ExplainMode\n\tswitch explainMode {\n\tcase utils.ExplainModeExtended:\n\t\tmode = proto.AnalyzePlanRequest_Explain_EXPLAIN_MODE_EXTENDED\n\tcase utils.ExplainModeSimple:\n\t\tmode = proto.AnalyzePlanRequest_Explain_EXPLAIN_MODE_SIMPLE\n\tcase utils.ExplainModeCost:\n\t\tmode = proto.AnalyzePlanRequest_Explain_EXPLAIN_MODE_COST\n\tcase utils.ExplainModeFormatted:\n\t\tmode = proto.AnalyzePlanRequest_Explain_EXPLAIN_MODE_FORMATTED\n\tcase utils.ExplainModeCodegen:\n\t\tmode = proto.AnalyzePlanRequest_Explain_EXPLAIN_MODE_CODEGEN\n\tdefault:\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"unsupported explain mode %v\",\n\t\t\texplainMode), sparkerrors.InvalidArgumentError)\n\t}\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_Explain_{\n\t\tExplain: &proto.AnalyzePlanRequest_Explain{\n\t\t\tPlan:        plan,\n\t\t\tExplainMode: mode,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn nil, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn response, nil\n}\n\nfunc (s *sparkConnectClientImpl) Persist(ctx context.Context, plan *proto.Plan, storageLevel utils.StorageLevel) error {\n\tprotoLevel := utils.ToProtoStorageLevel(storageLevel)\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_Persist_{\n\t\tPersist: &proto.AnalyzePlanRequest_Persist{\n\t\t\tRelation:     plan.GetRoot(),\n\t\t\tStorageLevel: protoLevel,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\t_, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn nil\n}\n\nfunc (s *sparkConnectClientImpl) Unpersist(ctx context.Context, plan *proto.Plan) error {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_Unpersist_{\n\t\tUnpersist: &proto.AnalyzePlanRequest_Unpersist{\n\t\t\tRelation: plan.GetRoot(),\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\t_, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn nil\n}\n\nfunc (s *sparkConnectClientImpl) GetStorageLevel(ctx context.Context, plan *proto.Plan) (*utils.StorageLevel, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_GetStorageLevel_{\n\t\tGetStorageLevel: &proto.AnalyzePlanRequest_GetStorageLevel{\n\t\t\tRelation: plan.GetRoot(),\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn nil, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\n\tlevel := response.GetGetStorageLevel().StorageLevel\n\tres := utils.FromProtoStorageLevel(level)\n\treturn &res, nil\n}\n\nfunc (s *sparkConnectClientImpl) SparkVersion(ctx context.Context) (string, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_SparkVersion_{\n\t\tSparkVersion: &proto.AnalyzePlanRequest_SparkVersion{},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn \"\", sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn response.GetSparkVersion().Version, nil\n}\n\nfunc (s *sparkConnectClientImpl) DDLParse(ctx context.Context, sql string) (*types.StructType, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_DdlParse{\n\t\tDdlParse: &proto.AnalyzePlanRequest_DDLParse{\n\t\t\tDdlString: sql,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn nil, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn types.ConvertProtoDataTypeToStructType(response.GetDdlParse().Parsed)\n}\n\nfunc (s *sparkConnectClientImpl) SameSemantics(ctx context.Context, plan1 *proto.Plan, plan2 *proto.Plan) (bool, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_SameSemantics_{\n\t\tSameSemantics: &proto.AnalyzePlanRequest_SameSemantics{\n\t\t\tTargetPlan: plan1,\n\t\t\tOtherPlan:  plan2,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn false, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn response.GetSameSemantics().GetResult(), nil\n}\n\nfunc (s *sparkConnectClientImpl) SemanticHash(ctx context.Context, plan *proto.Plan) (int32, error) {\n\trequest := s.newAnalyzePlanStub()\n\trequest.Analyze = &proto.AnalyzePlanRequest_SemanticHash_{\n\t\tSemanticHash: &proto.AnalyzePlanRequest_SemanticHash{\n\t\t\tPlan: plan,\n\t\t},\n\t}\n\t// Append the other items to the request.\n\tctx = metadata.NewOutgoingContext(ctx, s.metadata)\n\n\tresponse, err := s.client.AnalyzePlan(ctx, &request)\n\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\treturn 0, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t}\n\treturn response.GetSemanticHash().GetResult(), nil\n}\n\nfunc (s *sparkConnectClientImpl) Config(ctx context.Context,\n\toperation *proto.ConfigRequest_Operation,\n) (*proto.ConfigResponse, error) {\n\trequest := &proto.ConfigRequest{\n\t\tOperation: operation,\n\t\tUserContext: &proto.UserContext{\n\t\t\tUserId: s.opts.UserId,\n\t\t},\n\t\tClientType: &s.opts.UserAgent,\n\t}\n\trequest.SessionId = s.sessionId\n\tresp, err := s.client.Config(ctx, request)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn resp, nil\n}\n\nfunc NewSparkExecutor(conn *grpc.ClientConn, md metadata.MD, sessionId string, opts options.SparkClientOptions) base.SparkConnectClient {\n\tvar client base.SparkConnectRPCClient\n\tif opts.ReattachExecution {\n\t\tclient = NewRetriableSparkConnectClient(conn, sessionId, opts)\n\t} else {\n\t\tclient = proto.NewSparkConnectServiceClient(conn)\n\t}\n\treturn &sparkConnectClientImpl{\n\t\tclient:    client,\n\t\tmetadata:  md,\n\t\tsessionId: sessionId,\n\t\topts:      opts,\n\t}\n}\n\n// NewSparkExecutorFromClient creates a new SparkConnectClient from an existing client and is mostly\n// used in testing.\nfunc NewSparkExecutorFromClient(client base.SparkConnectRPCClient, md metadata.MD, sessionId string) base.SparkConnectClient {\n\treturn &sparkConnectClientImpl{\n\t\tclient:    client,\n\t\tmetadata:  md,\n\t\tsessionId: sessionId,\n\t\topts:      options.DefaultSparkClientOptions,\n\t}\n}\n\n// ExecutePlanClient is the wrapper around the result of the execution of a query plan using\n// Spark Connect.\ntype ExecutePlanClient struct {\n\t// The GRPC stream to read the response messages.\n\tresponseStream proto.SparkConnectService_ExecutePlanClient\n\t// The schema of the result of the operation.\n\tschema *types.StructType\n\t// The sessionId is ised to verify the server side session.\n\tsessionId  string\n\tdone       bool\n\tproperties map[string]any\n\topts       options.SparkClientOptions\n}\n\nfunc (c *ExecutePlanClient) Properties() map[string]any {\n\treturn c.properties\n}\n\n// ToTable converts the result of the execution of a query plan to an Arrow Table.\nfunc (c *ExecutePlanClient) ToTable() (*types.StructType, arrow.Table, error) {\n\tvar recordBatches []arrow.Record\n\tvar arrowSchema *arrow.Schema\n\trecordBatches = make([]arrow.Record, 0)\n\n\t// Explicitly needed when tracking re-attachble execution.\n\tc.done = false\n\tfor {\n\t\tresp, err := c.responseStream.Recv()\n\t\t// EOF is received when the last message has been processed and the stream\n\t\t// finished normally.\n\t\tif errors.Is(err, io.EOF) {\n\t\t\tbreak\n\t\t}\n\n\t\t// If the error was not EOF, there might be another error.\n\t\tif se := sparkerrors.FromRPCError(err); se != nil {\n\t\t\treturn nil, nil, sparkerrors.WithType(se, sparkerrors.ExecutionError)\n\t\t}\n\n\t\t// Process the message\n\n\t\t// Check that the server returned the session ID that we were expecting\n\t\t// and that it has not changed.\n\t\tif resp.GetSessionId() != c.sessionId {\n\t\t\treturn c.schema, nil, sparkerrors.WithType(&sparkerrors.InvalidServerSideSessionDetailsError{\n\t\t\t\tOwnSessionId:      c.sessionId,\n\t\t\t\tReceivedSessionId: resp.GetSessionId(),\n\t\t\t}, sparkerrors.InvalidServerSideSessionError)\n\t\t}\n\n\t\t// Check if the response has already the schema set and if yes, convert\n\t\t// the proto DataType to a StructType.\n\t\tif resp.Schema != nil {\n\t\t\tc.schema, err = types.ConvertProtoDataTypeToStructType(resp.Schema)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, nil, sparkerrors.WithType(err, sparkerrors.ExecutionError)\n\t\t\t}\n\t\t}\n\n\t\tswitch x := resp.ResponseType.(type) {\n\t\tcase *proto.ExecutePlanResponse_SqlCommandResult_:\n\t\t\tif val := x.SqlCommandResult.GetRelation(); val != nil {\n\t\t\t\tc.properties[\"sql_command_result\"] = val\n\t\t\t}\n\t\tcase *proto.ExecutePlanResponse_ArrowBatch_:\n\t\t\t// Do nothing.\n\t\t\trecord, err := types.ReadArrowBatchToRecord(x.ArrowBatch.Data, c.schema)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, nil, err\n\t\t\t}\n\t\t\tarrowSchema = record.Schema()\n\t\t\trecord.Retain()\n\t\t\trecordBatches = append(recordBatches, record)\n\t\tcase *proto.ExecutePlanResponse_ResultComplete_:\n\t\t\tc.done = true\n\t\tdefault:\n\t\t\t// Explicitly ignore messages that we cannot process at the moment.\n\t\t}\n\t}\n\n\t// Check that the result is logically complete. The result might not be complete\n\t// because after 2 minutes the server will interrupt the connection, and we have to\n\t// send a ReAttach execute request.\n\tif c.opts.ReattachExecution && !c.done {\n\t\treturn nil, nil, sparkerrors.WithType(fmt.Errorf(\"the result is not complete\"), sparkerrors.ExecutionError)\n\t}\n\t// Return the schema and table.\n\tif arrowSchema == nil {\n\t\treturn c.schema, nil, nil\n\t} else {\n\t\treturn c.schema, array.NewTableFromRecords(arrowSchema, recordBatches), nil\n\t}\n}\n\nfunc NewExecuteResponseStream(\n\tresponseClient proto.SparkConnectService_ExecutePlanClient,\n\tsessionId string,\n\toperationId string,\n\topts options.SparkClientOptions,\n) base.ExecuteResponseStream {\n\treturn &ExecutePlanClient{\n\t\tresponseStream: responseClient,\n\t\tsessionId:      sessionId,\n\t\tdone:           false,\n\t\tproperties:     make(map[string]any),\n\t\topts:           opts,\n\t}\n}\n\nfunc NewTestConnectClientFromResponses(sessionId string, r ...*mocks.MockResponse) base.SparkConnectClient {\n\tprotoClient := mocks.NewProtoClientMock(r...)\n\tstream := NewExecuteResponseStream(protoClient, sessionId, uuid.NewString(), options.DefaultSparkClientOptions)\n\treturn &mocks.TestExecutor{\n\t\tClient: stream,\n\t}\n}\n\nfunc NewTestConnectClientWithImmediateError(sessionId string, err error) base.SparkConnectClient {\n\tstream := NewExecuteResponseStream(nil, sessionId, uuid.NewString(), options.DefaultSparkClientOptions)\n\treturn &mocks.TestExecutor{\n\t\tClient: stream,\n\t\tErr:    err,\n\t}\n}\n"
  },
  {
    "path": "spark/client/client_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage client_test\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\t\"github.com/google/uuid\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/client\"\n\t\"github.com/apache/spark-connect-go/spark/client/testutils\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestAnalyzePlanCallsAnalyzePlanOnClient(t *testing.T) {\n\tctx := context.Background()\n\tresponse := &proto.AnalyzePlanResponse{}\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(nil, response, nil, nil), nil, mocks.MockSessionId)\n\tresp, err := c.AnalyzePlan(ctx, &proto.Plan{})\n\tassert.NoError(t, err)\n\tassert.NotNil(t, resp)\n}\n\nfunc TestAnalyzePlanFailsIfClientFails(t *testing.T) {\n\tctx := context.Background()\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(nil, nil, assert.AnError, nil), nil, mocks.MockSessionId)\n\tresp, err := c.AnalyzePlan(ctx, &proto.Plan{})\n\tassert.Nil(t, resp)\n\tassert.Error(t, err)\n}\n\nfunc TestExecutePlanCallsExecutePlanOnClient(t *testing.T) {\n\tctx := context.Background()\n\tplan := &proto.Plan{}\n\n\t// Generate a mock client\n\tresponseStream := mocks.NewProtoClientMock(&mocks.ExecutePlanResponseDone)\n\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(responseStream, nil, nil, t), nil, mocks.MockSessionId)\n\tresp, err := c.ExecutePlan(ctx, plan)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, resp)\n}\n\nfunc TestExecutePlanCallsExecuteCommandOnClient(t *testing.T) {\n\tctx := context.Background()\n\tplan := &proto.Plan{}\n\n\t// Generate a mock client\n\tresponseStream := mocks.NewProtoClientMock(&mocks.ExecutePlanResponseDone, &mocks.ExecutePlanResponseEOF)\n\n\t// Check that the execution fails if no command is supplied.\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(responseStream, nil, nil, t), nil, mocks.MockSessionId)\n\t_, _, _, err := c.ExecuteCommand(ctx, plan)\n\tassert.ErrorIs(t, err, sparkerrors.ExecutionError)\n\n\t// Generate a command and the execution should succeed.\n\tsqlCommand := mocks.NewSqlCommand(\"select range(10)\")\n\tc = client.NewSparkExecutorFromClient(testutils.NewConnectServiceClientMock(responseStream, nil, nil, t), nil, mocks.MockSessionId)\n\t_, _, _, err = c.ExecuteCommand(ctx, sqlCommand)\n\tassert.NoError(t, err)\n}\n\nfunc Test_ExecuteWithWrongSession(t *testing.T) {\n\tctx := context.Background()\n\tsqlCommand := mocks.NewSqlCommand(\"select range(10)\")\n\n\t// Generate a mock client\n\tresponseStream := mocks.NewProtoClientMock(&mocks.ExecutePlanResponseDone, &mocks.ExecutePlanResponseEOF)\n\n\t// Check that the execution fails if no command is supplied.\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(responseStream, nil, nil, t), nil, uuid.NewString())\n\t_, _, _, err := c.ExecuteCommand(ctx, sqlCommand)\n\tassert.ErrorIs(t, err, sparkerrors.InvalidServerSideSessionError)\n}\n\nfunc Test_Execute_SchemaParsingFails(t *testing.T) {\n\tctx := context.Background()\n\tsqlCommand := mocks.NewSqlCommand(\"select range(10)\")\n\tresponseStream := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseBrokenSchema,\n\t\t&mocks.ExecutePlanResponseDone,\n\t\t&mocks.ExecutePlanResponseEOF)\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(responseStream, nil, nil, t), nil, mocks.MockSessionId)\n\t_, _, _, err := c.ExecuteCommand(ctx, sqlCommand)\n\tassert.ErrorIs(t, err, sparkerrors.ExecutionError)\n}\n"
  },
  {
    "path": "spark/client/conf.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage client\n\nimport (\n\t\"context\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/client/base\"\n)\n\n// Public interface RuntimeConfig\ntype RuntimeConfig interface {\n\tGetAll(ctx context.Context) (map[string]string, error)\n\tSet(ctx context.Context, key string, value string) error\n\tGet(ctx context.Context, key string) (string, error)\n\tUnset(ctx context.Context, key string) error\n\tIsModifiable(ctx context.Context, key string) (bool, error)\n\tGetWithDefault(ctx context.Context, key string, default_value string) (string, error)\n}\n\n// private type with private member client\ntype runtimeConfig struct {\n\tclient *base.SparkConnectClient\n}\n\n// GetAll returns all configured keys in a map of strings\nfunc (r runtimeConfig) GetAll(ctx context.Context) (map[string]string, error) {\n\treq := &proto.ConfigRequest_GetAll{}\n\toperation := &proto.ConfigRequest_Operation_GetAll{GetAll: req}\n\top := &proto.ConfigRequest_Operation{OpType: operation}\n\tresp, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tm := make(map[string]string, 0)\n\tfor _, k := range resp.GetPairs() {\n\t\tif k.Value != nil {\n\t\t\tm[k.Key] = *k.Value\n\t\t}\n\t}\n\treturn m, nil\n}\n\n// Set takes a key and a value and sets it in the config\nfunc (r runtimeConfig) Set(ctx context.Context, key string, value string) error {\n\treqArr := []*proto.KeyValue{{Key: key, Value: &value}}\n\treq := &proto.ConfigRequest_Set{\n\t\tPairs: reqArr,\n\t}\n\top := &proto.ConfigRequest_Operation{OpType: &proto.ConfigRequest_Operation_Set{Set: req}}\n\t_, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn err\n\t}\n\treturn nil\n}\n\nfunc (r runtimeConfig) Get(ctx context.Context, key string) (string, error) {\n\treq := &proto.ConfigRequest_Get{Keys: []string{key}}\n\toperation := &proto.ConfigRequest_Operation_Get{Get: req}\n\top := &proto.ConfigRequest_Operation{OpType: operation}\n\tresp, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn \"\", err\n\t}\n\treturn *resp.GetPairs()[0].Value, nil\n}\n\nfunc (r runtimeConfig) Unset(ctx context.Context, key string) error {\n\treq := &proto.ConfigRequest_Unset{Keys: []string{key}}\n\toperation := &proto.ConfigRequest_Operation_Unset{Unset: req}\n\top := &proto.ConfigRequest_Operation{OpType: operation}\n\t_, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn err\n\t}\n\treturn nil\n}\n\nfunc (r runtimeConfig) IsModifiable(ctx context.Context, key string) (bool, error) {\n\treq := &proto.ConfigRequest_IsModifiable{Keys: []string{key}}\n\toperation := &proto.ConfigRequest_Operation_IsModifiable{IsModifiable: req}\n\top := &proto.ConfigRequest_Operation{OpType: operation}\n\tresp, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn false, err\n\t}\n\tre := *resp.GetPairs()[0].Value\n\tif re == \"true\" {\n\t\treturn true, nil\n\t} else {\n\t\treturn false, nil\n\t}\n}\n\nfunc (r runtimeConfig) GetWithDefault(ctx context.Context, key string, default_value string) (string, error) {\n\tp := make([]*proto.KeyValue, 0)\n\tp = append(p, &proto.KeyValue{Key: key, Value: &default_value})\n\treq := &proto.ConfigRequest_GetWithDefault{Pairs: p}\n\toperation := &proto.ConfigRequest_Operation_GetWithDefault{GetWithDefault: req}\n\top := &proto.ConfigRequest_Operation{OpType: operation}\n\tresp, err := (*r.client).Config(ctx, op)\n\tif err != nil {\n\t\treturn \"\", err\n\t}\n\n\treturn *resp.GetPairs()[0].Value, nil\n}\n\n// Constructor for runtimeConfig used by SparkSession\nfunc NewRuntimeConfig(client *base.SparkConnectClient) *runtimeConfig {\n\treturn &runtimeConfig{client: client}\n}\n"
  },
  {
    "path": "spark/client/options/options.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage options\n\ntype SparkClientOptions struct {\n\tReattachExecution bool\n\tUserAgent         string\n\tUserId            string\n}\n\nvar DefaultSparkClientOptions = SparkClientOptions{\n\tReattachExecution: false,\n}\n\nfunc NewSparkClientOptions(reattach bool) SparkClientOptions {\n\treturn SparkClientOptions{\n\t\tReattachExecution: reattach,\n\t}\n}\n"
  },
  {
    "path": "spark/client/retry.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage client\n\nimport (\n\t\"context\"\n\t\"errors\"\n\t\"io\"\n\t\"math/rand\"\n\t\"strings\"\n\t\"time\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/base\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/options\"\n\t\"google.golang.org/grpc/metadata\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"google.golang.org/grpc\"\n\t\"google.golang.org/grpc/codes\"\n)\n\ntype RetryHandler func(error) bool\n\n// RetryPolicy defines the parameters for a retry policy. The policy is used to determine if an\n// error is retriable and how to handle retries. The policy defines the behavior of the client\n// in how it backs off in case of an error and how the retries are spread out over time.\ntype RetryPolicy struct {\n\tMaxRetries         int32\n\tInitialBackoff     time.Duration\n\tMaxBackoff         time.Duration\n\tBackoffMultiplier  float32\n\tJitter             time.Duration\n\tMinJitterThreshold time.Duration\n\tName               string\n\tHandler            RetryHandler\n}\n\n// DefaultRetryPolicy is the default retry policy used by the client. It will retry on Unavailable and\n// in case the cursor has been disconnected. All other errors are considered to be not retriable.\nvar DefaultRetryPolicy = RetryPolicy{\n\tMaxRetries:         15,\n\tInitialBackoff:     50 * time.Millisecond,\n\tMaxBackoff:         1 * time.Minute,\n\tBackoffMultiplier:  4.0,\n\tJitter:             500 * time.Millisecond,\n\tMinJitterThreshold: 2000 * time.Millisecond,\n\tName:               \"DefaultRetryPolicy\",\n\tHandler: func(e error) bool {\n\t\tstatus := sparkerrors.FromRPCError(e)\n\t\tswitch status.Code {\n\t\tcase codes.Unavailable:\n\t\t\treturn true\n\t\tcase codes.Internal:\n\t\t\tif strings.Contains(status.Message, \"INVALID_CURSOR.DISCONNECTED\") {\n\t\t\t\treturn true\n\t\t\t}\n\t\t}\n\t\treturn false\n\t},\n}\n\nvar TestingRetryPolicy = RetryPolicy{\n\tMaxRetries:         5,\n\tInitialBackoff:     0,\n\tMaxBackoff:         1,\n\tBackoffMultiplier:  2,\n\tJitter:             0,\n\tMinJitterThreshold: 0,\n\tName:               \"TestingRetryPolicy\",\n\tHandler: func(e error) bool {\n\t\tstatus := sparkerrors.FromRPCError(e)\n\t\tswitch status.Code {\n\t\tcase codes.Unavailable:\n\t\t\treturn true\n\t\tcase codes.Internal:\n\t\t\tif strings.Contains(status.Message, \"INVALID_CURSOR.DISCONNECTED\") {\n\t\t\t\treturn true\n\t\t\t}\n\t\t}\n\t\treturn false\n\t},\n}\n\n// DefaultRetryPolicyRegistry is the default set of retry policies used by the client. It contains\n// all those policies that are enabled by default.\nvar DefaultRetryPolicyRegistry = []RetryPolicy{DefaultRetryPolicy}\n\n// retryState is the current state of the retries for one particular RPC request. The retry\n// state is independent of the retry policy.\ntype retryState struct {\n\tretryCount int32\n\tnextWait   time.Duration\n}\n\n// nextAttempt calculates the next wait time for the next retry attempt. The function returns\n// nil if the maximum number of retries has been exceeded, otherwise it returns the amount\n// of time the caller should wait.\nfunc (rs *retryState) nextAttempt(p RetryPolicy) *time.Duration {\n\tif rs.retryCount >= p.MaxRetries {\n\t\treturn nil\n\t}\n\n\t// For the first retry pick the initial backoff of the matching policy.\n\tif rs.retryCount == 0 {\n\t\trs.nextWait = p.InitialBackoff\n\t}\n\n\t// Adjust the retry count and calculate the next wait.\n\trs.retryCount++\n\twait := rs.nextWait\n\trs.nextWait = time.Duration(float32(rs.nextWait.Milliseconds())*p.BackoffMultiplier) * time.Millisecond\n\tif rs.nextWait > p.MaxBackoff {\n\t\trs.nextWait = p.MaxBackoff\n\t}\n\n\t// Some policies define that jitter should only be applied after a particular threshold.\n\tif wait > p.MinJitterThreshold {\n\t\twait += time.Duration(rand.Float32() * float32(p.Jitter.Milliseconds()))\n\t}\n\n\treturn &wait\n}\n\nfunc NewRetriableSparkConnectClient(conn *grpc.ClientConn, sessionId string,\n\topts options.SparkClientOptions,\n) base.SparkConnectRPCClient {\n\tinnerClient := proto.NewSparkConnectServiceClient(conn)\n\treturn &retriableSparkConnectClient{\n\t\tclient:        innerClient,\n\t\tsessionId:     sessionId,\n\t\tretryPolicies: DefaultRetryPolicyRegistry,\n\t\toptions:       opts,\n\t}\n}\n\n// wrapRetriableCall wraps a call to a function that returns a result and an error. The function is\n// retried according to the retry policies. The function will return the result or an error if the\n// retries are exceeded.\nfunc wrapRetriableCall[Res rpcType](ctx context.Context, retryPolicies []RetryPolicy, in func(context.Context) (Res, error)) (Res, error) {\n\tvar lastErr error\n\tvar response Res\n\t// Create the retry state for this wrapped call. The retry state captures the information about\n\t// the wait time and how many retries to perform.\n\tstate := retryState{}\n\t// As long as the error is retriable, we will retry the operation.\n\tcanRetry := true\n\tfor canRetry {\n\t\t// Every loop iteration starts with being non-retriable.\n\t\tcanRetry = false\n\t\tresponse, lastErr = in(ctx)\n\t\tif lastErr != nil {\n\t\t\tfor _, h := range retryPolicies {\n\t\t\t\tif h.Handler(lastErr) {\n\t\t\t\t\tcanRetry = true\n\t\t\t\t\twait := state.nextAttempt(h)\n\t\t\t\t\tif wait != nil {\n\t\t\t\t\t\ttime.Sleep(*wait)\n\t\t\t\t\t} else {\n\t\t\t\t\t\t// If the retries are exceeded, simply return from here.\n\t\t\t\t\t\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n\t\t\t\t\t}\n\t\t\t\t\t// Breaks out of the retry handler loop.\n\t\t\t\t\tbreak\n\t\t\t\t}\n\t\t\t}\n\t\t} else {\n\t\t\t// Exit loop if no error has been received.\n\t\t\treturn response, nil\n\t\t}\n\t}\n\t// TODO: Should this simoly return the original error?\n\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n}\n\ntype rpcType interface {\n\t*proto.AnalyzePlanResponse | *proto.ConfigResponse | *proto.ArtifactStatusesResponse |\n\t\t*proto.InterruptResponse | *proto.ReleaseExecuteResponse | *proto.ExecutePlanResponse |\n\t\t*proto.ReleaseSessionResponse | *proto.FetchErrorDetailsResponse\n}\n\n// retriableSparkConnectClient wraps the SparkConnectServiceClient implementation to\n// transparently handle retries.\ntype retriableSparkConnectClient struct {\n\tclient    base.SparkConnectRPCClient\n\tsessionId string\n\t// Not yet used.\n\t// serverSideSessionId string\n\tretryPolicies []RetryPolicy\n\toptions       options.SparkClientOptions\n}\n\n// FetchErrorDetails implements base.SparkConnectRPCClient.\nfunc (r *retriableSparkConnectClient) FetchErrorDetails(ctx context.Context,\n\tin *proto.FetchErrorDetailsRequest, opts ...grpc.CallOption,\n) (*proto.FetchErrorDetailsResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (\n\t\t*proto.FetchErrorDetailsResponse, error,\n\t) {\n\t\treturn r.client.FetchErrorDetails(ctx2, in, opts...)\n\t})\n}\n\n// ReleaseSession implements base.SparkConnectRPCClient.\nfunc (r *retriableSparkConnectClient) ReleaseSession(ctx context.Context,\n\tin *proto.ReleaseSessionRequest, opts ...grpc.CallOption,\n) (*proto.ReleaseSessionResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (*proto.ReleaseSessionResponse, error) {\n\t\treturn r.client.ReleaseSession(ctx2, in, opts...)\n\t})\n}\n\nfunc (r *retriableSparkConnectClient) ExecutePlan(ctx context.Context, in *proto.ExecutePlanRequest,\n\topts ...grpc.CallOption,\n) (proto.SparkConnectService_ExecutePlanClient, error) {\n\tvar lastErr error\n\t// Create the retry state for this wrapped call. The retry state captures the information about\n\t// the wait time and how many retries to perform.\n\tstate := retryState{}\n\t// As long as the error is retriable, we will retry the operation.\n\tcanRetry := true\n\tfor canRetry {\n\t\t// Every loop iteration starts with being non-retriable.\n\t\tcanRetry = false\n\t\tresponse, lastErr := r.client.ExecutePlan(ctx, in, opts...)\n\t\tif lastErr != nil {\n\t\t\tfor _, h := range r.retryPolicies {\n\t\t\t\tif h.Handler(lastErr) {\n\t\t\t\t\tcanRetry = true\n\t\t\t\t\twait := state.nextAttempt(h)\n\t\t\t\t\tif wait != nil {\n\t\t\t\t\t\ttime.Sleep(*wait)\n\t\t\t\t\t} else {\n\t\t\t\t\t\t// If the retries are exceeded, simply return from here.\n\t\t\t\t\t\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n\t\t\t\t\t}\n\t\t\t\t\t// Breaks out of the retry handler loop.\n\t\t\t\t\tbreak\n\t\t\t\t}\n\t\t\t}\n\t\t} else {\n\t\t\t// Exit loop if no error has been received.\n\t\t\trc := retriableExecutePlanClient{\n\t\t\t\tcontext: ctx,\n\t\t\t\tretryContext: &retryContext{\n\t\t\t\t\tstream:         response,\n\t\t\t\t\tclient:         r,\n\t\t\t\t\trequest:        in,\n\t\t\t\t\tresultComplete: false,\n\t\t\t\t\tretryPolicies:  r.retryPolicies,\n\t\t\t\t},\n\t\t\t}\n\t\t\treturn rc, nil\n\t\t}\n\t}\n\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n}\n\nfunc (r *retriableSparkConnectClient) AnalyzePlan(ctx context.Context, in *proto.AnalyzePlanRequest,\n\topts ...grpc.CallOption,\n) (*proto.AnalyzePlanResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (*proto.AnalyzePlanResponse, error) {\n\t\treturn r.client.AnalyzePlan(ctx2, in, opts...)\n\t})\n}\n\nfunc (r *retriableSparkConnectClient) Config(ctx context.Context, in *proto.ConfigRequest, opts ...grpc.CallOption) (*proto.ConfigResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (*proto.ConfigResponse, error) {\n\t\treturn r.client.Config(ctx2, in, opts...)\n\t})\n}\n\nfunc (r *retriableSparkConnectClient) AddArtifacts(ctx context.Context, opts ...grpc.CallOption) (proto.SparkConnectService_AddArtifactsClient, error) {\n\tvar lastErr error\n\t// Create the retry state for this wrapped call. The retry state captures the information about\n\t// the wait time and how many retries to perform.\n\tstate := retryState{}\n\t// As long as the error is retriable, we will retry the operation.\n\tcanRetry := true\n\tfor canRetry {\n\t\t// Every loop iteration starts with being non-retriable.\n\t\tcanRetry = false\n\t\tresponse, lastErr := r.client.AddArtifacts(ctx, opts...)\n\t\tif lastErr != nil {\n\t\t\tfor _, h := range r.retryPolicies {\n\t\t\t\tif h.Handler(lastErr) {\n\t\t\t\t\tcanRetry = true\n\t\t\t\t\twait := state.nextAttempt(h)\n\t\t\t\t\tif wait != nil {\n\t\t\t\t\t\ttime.Sleep(*wait)\n\t\t\t\t\t} else {\n\t\t\t\t\t\t// If the retries are exceeded, simply return from here.\n\t\t\t\t\t\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n\t\t\t\t\t}\n\t\t\t\t\t// Breaks out of the retry handler loop.\n\t\t\t\t\tbreak\n\t\t\t\t}\n\t\t\t}\n\t\t} else {\n\t\t\t// Exit loop if no error has been received.\n\t\t\treturn response, nil\n\t\t}\n\t}\n\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n}\n\nfunc (r *retriableSparkConnectClient) ArtifactStatus(ctx context.Context,\n\tin *proto.ArtifactStatusesRequest, opts ...grpc.CallOption,\n) (*proto.ArtifactStatusesResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (\n\t\t*proto.ArtifactStatusesResponse, error,\n\t) {\n\t\treturn r.client.ArtifactStatus(ctx2, in, opts...)\n\t})\n}\n\nfunc (r *retriableSparkConnectClient) Interrupt(ctx context.Context, in *proto.InterruptRequest,\n\topts ...grpc.CallOption,\n) (*proto.InterruptResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (*proto.InterruptResponse, error) {\n\t\treturn r.client.Interrupt(ctx2, in, opts...)\n\t})\n}\n\nfunc (r *retriableSparkConnectClient) ReattachExecute(ctx context.Context,\n\tin *proto.ReattachExecuteRequest, opts ...grpc.CallOption,\n) (proto.SparkConnectService_ReattachExecuteClient, error) {\n\tvar lastErr error\n\t// Create the retry state for this wrapped call. The retry state captures the information about\n\t// the wait time and how many retries to perform.\n\tstate := retryState{}\n\t// As long as the error is retriable, we will retry the operation.\n\tcanRetry := true\n\tfor canRetry {\n\t\t// Every loop iteration starts with being non-retriable.\n\t\tcanRetry = false\n\t\tresponse, lastErr := r.client.ReattachExecute(ctx, in, opts...)\n\t\tif lastErr != nil {\n\t\t\tfor _, h := range r.retryPolicies {\n\t\t\t\tif h.Handler(lastErr) {\n\t\t\t\t\tcanRetry = true\n\t\t\t\t\twait := state.nextAttempt(h)\n\t\t\t\t\tif wait != nil {\n\t\t\t\t\t\ttime.Sleep(*wait)\n\t\t\t\t\t} else {\n\t\t\t\t\t\t// If the retries are exceeded, simply return from here.\n\t\t\t\t\t\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n\t\t\t\t\t}\n\t\t\t\t\t// Breaks out of the retry handler loop.\n\t\t\t\t\tbreak\n\t\t\t\t}\n\t\t\t}\n\t\t} else {\n\t\t\t// Exit loop if no error has been received.\n\t\t\t// TODO: Re-attaching needs to be retriable as well.\n\t\t\treturn response, nil\n\t\t}\n\t}\n\treturn nil, sparkerrors.WithType(lastErr, sparkerrors.RetriesExceeded)\n}\n\nfunc (r *retriableSparkConnectClient) ReleaseExecute(ctx context.Context,\n\tin *proto.ReleaseExecuteRequest, opts ...grpc.CallOption,\n) (*proto.ReleaseExecuteResponse, error) {\n\treturn wrapRetriableCall(ctx, r.retryPolicies, func(ctx2 context.Context) (*proto.ReleaseExecuteResponse, error) {\n\t\treturn r.client.ReleaseExecute(ctx2, in, opts...)\n\t})\n}\n\ntype retryContext struct {\n\tstream         proto.SparkConnectService_ExecutePlanClient\n\tclient         base.SparkConnectRPCClient\n\trequest        *proto.ExecutePlanRequest\n\tlastResponseId *string\n\tresultComplete bool\n\tretryPolicies  []RetryPolicy\n}\n\n// retriableExecutePlanClient is a wrapper around the ExecutePlanClient that handles retries\n// transparently. Since the interface has to follow the ExecutePlanClient interface, we have to\n// implement all methods of the interface and follow their method receiver pattern. As the main\n// methods do not implement a pointer receiver we're wrapping the variable part of the retry\n// behahivor in a separate struct.\n//\n// In addition, we capture the original Context of the caller that is passed to the interface. While\n// this is typically not a desired pattern it is the only way to make sure the same context is used\n// across the retrying and underlying struct.\ntype retriableExecutePlanClient struct {\n\tretryContext *retryContext\n\tcontext      context.Context\n}\n\nfunc (r retriableExecutePlanClient) Recv() (*proto.ExecutePlanResponse, error) {\n\treturn wrapRetriableCall(r.context, r.retryContext.retryPolicies, func(ctx2 context.Context) (*proto.ExecutePlanResponse, error) {\n\t\tresp, err := r.retryContext.stream.Recv()\n\t\t// Success, simply return the result.\n\t\tif err == nil {\n\t\t\tr.retryContext.lastResponseId = &resp.ResponseId\n\t\t\treturn resp, nil\n\t\t}\n\t\t// Ignore successful closure.\n\t\tif errors.Is(err, io.EOF) {\n\t\t\treturn nil, err\n\t\t}\n\t\t// Now we have to assume that the request has failed, and we distinguish two cases: First, we have\n\t\t// never received a result and in this case we simply execute the same request again. Second,\n\t\t// we will send a reattach request with the same operation ID and the last response ID.\n\t\tif r.retryContext.lastResponseId == nil {\n\t\t\t// Send the request again.\n\t\t\trs, execErr := r.retryContext.client.ExecutePlan(ctx2, r.retryContext.request)\n\t\t\tif execErr != nil {\n\t\t\t\treturn nil, execErr\n\t\t\t}\n\t\t\tswitch stream := rs.(type) {\n\t\t\tcase retriableExecutePlanClient:\n\t\t\t\tr.retryContext.stream = stream.retryContext.stream\n\t\t\tdefault:\n\t\t\t\tr.retryContext.stream = stream\n\t\t\t}\n\t\t\treturn nil, err\n\t\t} else {\n\t\t\t// Send a reattach\n\t\t\treq := &proto.ReattachExecuteRequest{\n\t\t\t\tSessionId:      r.retryContext.request.SessionId,\n\t\t\t\tUserContext:    r.retryContext.request.UserContext,\n\t\t\t\tOperationId:    *r.retryContext.request.OperationId,\n\t\t\t\tLastResponseId: r.retryContext.lastResponseId,\n\t\t\t}\n\t\t\tre, execErr := r.retryContext.client.ReattachExecute(ctx2, req)\n\t\t\tif execErr != nil {\n\t\t\t\treturn nil, execErr\n\t\t\t}\n\t\t\tswitch stream := re.(type) {\n\t\t\tcase retriableExecutePlanClient:\n\t\t\t\tr.retryContext.stream = stream.retryContext.stream\n\t\t\tdefault:\n\t\t\t\tr.retryContext.stream = stream\n\t\t\t}\n\t\t\treturn nil, err\n\t\t}\n\t})\n}\n\nfunc (r retriableExecutePlanClient) Header() (metadata.MD, error) {\n\treturn r.retryContext.stream.Header()\n}\n\nfunc (r retriableExecutePlanClient) Trailer() metadata.MD {\n\treturn r.retryContext.stream.Trailer()\n}\n\nfunc (r retriableExecutePlanClient) CloseSend() error {\n\treturn r.retryContext.stream.CloseSend()\n}\n\nfunc (r retriableExecutePlanClient) Context() context.Context {\n\treturn r.retryContext.stream.Context()\n}\n\nfunc (r retriableExecutePlanClient) SendMsg(m any) error {\n\treturn r.retryContext.stream.SendMsg(m)\n}\n\nfunc (r retriableExecutePlanClient) RecvMsg(m any) error {\n\treturn r.retryContext.stream.RecvMsg(m)\n}\n"
  },
  {
    "path": "spark/client/retry_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage client\n\nimport (\n\t\"context\"\n\t\"errors\"\n\t\"io\"\n\t\"testing\"\n\t\"time\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/options\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/testutils\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\n\t\"github.com/stretchr/testify/assert\"\n\t\"google.golang.org/grpc/codes\"\n\t\"google.golang.org/grpc/status\"\n)\n\nfunc alwaysRetry(err error) bool {\n\treturn true\n}\n\nfunc Test_retryState_nextAttempt(t *testing.T) {\n\tbasePolicy := RetryPolicy{\n\t\tMaxRetries:         15,\n\t\tMaxBackoff:         60 * time.Second,\n\t\tInitialBackoff:     100 * time.Millisecond,\n\t\tBackoffMultiplier:  4,\n\t\tJitter:             time.Millisecond * 100,\n\t\tMinJitterThreshold: 2 * time.Second,\n\t\tName:               \"BasePolicy\",\n\t\tHandler:            alwaysRetry,\n\t}\n\n\ttype fields struct {\n\t\tretryCount int32\n\t\tnextWait   time.Duration\n\t}\n\ttype args struct {\n\t\tp RetryPolicy\n\t}\n\ttests := []struct {\n\t\tname      string\n\t\tfields    fields\n\t\targs      args\n\t\twantLower time.Duration\n\t\twantUpper time.Duration\n\t\texceeded  bool\n\t}{\n\t\t{\n\t\t\t\"BasicRetry - initial backoff\",\n\t\t\tfields{\n\t\t\t\t0,\n\t\t\t\t0,\n\t\t\t},\n\t\t\targs{\n\t\t\t\tbasePolicy,\n\t\t\t},\n\t\t\t100 * time.Millisecond,\n\t\t\t0,\n\t\t\tfalse,\n\t\t},\n\t\t{\n\t\t\t\"Jitter applied correctly\",\n\t\t\tfields{\n\t\t\t\t1,\n\t\t\t\t3 * time.Second,\n\t\t\t},\n\t\t\targs{\n\t\t\t\tbasePolicy,\n\t\t\t},\n\t\t\t3 * time.Second,\n\t\t\t3*time.Second + basePolicy.Jitter,\n\t\t\tfalse,\n\t\t},\n\n\t\t{\n\t\t\t\"Retries Exceeded\",\n\t\t\tfields{\n\t\t\t\t16,\n\t\t\t\t0,\n\t\t\t},\n\t\t\targs{\n\t\t\t\tbasePolicy,\n\t\t\t},\n\t\t\t0,\n\t\t\t0,\n\t\t\ttrue,\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\trs := &retryState{\n\t\t\t\tretryCount: tt.fields.retryCount,\n\t\t\t\tnextWait:   tt.fields.nextWait,\n\t\t\t}\n\t\t\tif tt.exceeded {\n\t\t\t\tassert.Nilf(t, rs.nextAttempt(tt.args.p),\n\t\t\t\t\t\"Expecting retries to be exceeded (%v, %v)\", rs, tt.args.p)\n\t\t\t} else {\n\t\t\t\tval := *rs.nextAttempt(tt.args.p)\n\t\t\t\tif tt.wantUpper > 0 {\n\t\t\t\t\tassert.LessOrEqualf(t, tt.wantLower, val, \"nextAttempt(%v, %v)\", rs, tt.args.p)\n\t\t\t\t\tassert.GreaterOrEqualf(t, tt.wantUpper, val, \"nextAttempt(%v, %v)\", rs, tt.args.p)\n\t\t\t\t} else {\n\t\t\t\t\tassert.Equalf(t, tt.wantLower, val, \"nextAttempt(%v, %v)\", rs, tt.args.p)\n\t\t\t\t}\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc Test_retryMaxBackOff_applied(t *testing.T) {\n\tbasePolicy := RetryPolicy{\n\t\tMaxRetries:         15,\n\t\tMaxBackoff:         60 * time.Second,\n\t\tInitialBackoff:     100 * time.Millisecond,\n\t\tBackoffMultiplier:  4,\n\t\tJitter:             time.Millisecond * 100,\n\t\tMinJitterThreshold: 2 * time.Second,\n\t\tName:               \"BasePolicy\",\n\t\tHandler:            alwaysRetry,\n\t}\n\n\tstate := retryState{\n\t\tretryCount: 3,\n\t\tnextWait:   30 * time.Second,\n\t}\n\n\twait := state.nextAttempt(basePolicy)\n\tassert.LessOrEqualf(t, 30*time.Second, *wait, \" nowWait: nextAttempt(%v, %v)\", state, basePolicy)\n\tassert.GreaterOrEqualf(t, 30*time.Second+basePolicy.Jitter, *wait,\n\t\t\" nowWait: nextAttempt(%v, %v)\", state, basePolicy)\n\tassert.Equalf(t, 60*time.Second, state.nextWait, \" nextWait: nextAttempt(%v, %v)\", state, basePolicy)\n}\n\nfunc Test_defaultRetryPolicyBehavior(t *testing.T) {\n\tstate := retryState{\n\t\t0,\n\t\t0,\n\t}\n\n\tw := state.nextAttempt(DefaultRetryPolicy)\n\tassert.NotNil(t, w)\n\tassert.Equal(t, DefaultRetryPolicy.InitialBackoff, *w)\n\n\t// Check the next iterations until failure\n\tw = state.nextAttempt(DefaultRetryPolicy)\n\tassert.NotNil(t, w)\n\texpected := time.Duration(int64(float32(DefaultRetryPolicy.InitialBackoff.Milliseconds())*\n\t\tDefaultRetryPolicy.BackoffMultiplier)) * time.Millisecond\n\tassert.GreaterOrEqual(t, expected, *w)\n\n\tfor i := int32(2); i < DefaultRetryPolicy.MaxRetries; i++ {\n\t\tw = state.nextAttempt(DefaultRetryPolicy)\n\t\tassert.NotNil(t, w)\n\t\tassert.LessOrEqualf(t, *w, 60*time.Second+DefaultRetryPolicy.Jitter,\n\t\t\t\"nextAttempt(%v, %v)\", state, DefaultRetryPolicy)\n\t}\n\t// Check that the next attempt is nil\n\tw = state.nextAttempt(DefaultRetryPolicy)\n\tassert.Nil(t, w)\n}\n\nfunc Test_default_retryHandler(t *testing.T) {\n\terr := io.EOF\n\tassert.Falsef(t, DefaultRetryPolicy.Handler(err), \"Must not retry other errors\")\n\terr = errors.New(\"Some error\")\n\tassert.Falsef(t, DefaultRetryPolicy.Handler(err), \"Must not retry other errors\")\n\ts := status.New(codes.Unavailable, \"Unavailable\")\n\tassert.Truef(t, DefaultRetryPolicy.Handler(s.Err()), \"Must retry Unavailable\")\n\ts = status.New(codes.Internal, \"ANALYSIS EXCEPTION\")\n\tassert.Falsef(t, DefaultRetryPolicy.Handler(s.Err()), \"Must not retry Internal\")\n\ts = status.New(codes.Internal, \"Error: INVALID_CURSOR.DISCONNECTED\")\n\tassert.Truef(t, DefaultRetryPolicy.Handler(s.Err()),\n\t\t\"Must retry Internal with INVALID_CURSOR.DISCONNECTED\")\n}\n\nfunc Test_retriable_success(t *testing.T) {\n\ttoRetry := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t)\n\tresponseStream := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseDone,\n\t\t&mocks.ExecutePlanResponseEOF)\n\n\tc := testutils.NewConnectServiceClientMock(responseStream, nil, nil, t)\n\tstream := retriableExecutePlanClient{\n\t\tcontext: context.Background(),\n\t\tretryContext: &retryContext{\n\t\t\tstream:        toRetry,\n\t\t\tclient:        c,\n\t\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\t},\n\t}\n\t_, err := stream.Recv()\n\tassert.NoError(t, err)\n}\n\nfunc Test_retriable_client(t *testing.T) {\n\ttoRetry := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t)\n\n\tresponseStream := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseDone,\n\t\t&mocks.ExecutePlanResponseEOF)\n\n\tc := testutils.NewConnectServiceClientMock(responseStream, nil, nil, t)\n\tstream := retriableExecutePlanClient{\n\t\tcontext: context.Background(),\n\t\tretryContext: &retryContext{\n\t\t\tstream: toRetry,\n\t\t\tclient: c,\n\t\t},\n\t}\n\n\t_, err := stream.Recv()\n\tassert.ErrorIs(t, err, sparkerrors.RetriesExceeded)\n\n\tc = testutils.NewConnectServiceClientMock(toRetry, nil, nil, t)\n\tstream = retriableExecutePlanClient{\n\t\tcontext: context.Background(),\n\t\tretryContext: &retryContext{\n\t\t\tstream:        toRetry,\n\t\t\tclient:        c,\n\t\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\t},\n\t}\n\n\t_, err = stream.Recv()\n\tassert.ErrorIs(t, err, sparkerrors.RetriesExceeded)\n}\n\nfunc Test_retriable_with_reattach(t *testing.T) {\n\t//\n\ttoRetry := mocks.NewProtoClientMock(\n\t\t&mocks.ExecutePlanResponseWithSchema,\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t)\n\n\t// Final response stream.\n\tresponseStream := mocks.NewProtoClientMock(\n\t\t// First let's do another round of retry and then complete.\n\t\t&mocks.ExecutePlanResponseUnavailable,\n\t\t// Now, finsih the stream successfully\n\t\t&mocks.ExecutePlanResponseDone,\n\t\t&mocks.ExecutePlanResponseEOF)\n\n\tc := testutils.NewConnectServiceClientMock(responseStream, nil, nil, t)\n\tclient := retriableSparkConnectClient{\n\t\tclient:        c,\n\t\tsessionId:     mocks.MockSessionId,\n\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\toptions:       options.DefaultSparkClientOptions,\n\t}\n\n\tstream := retriableExecutePlanClient{\n\t\tcontext: context.Background(),\n\t\tretryContext: &retryContext{\n\t\t\tstream:        toRetry,\n\t\t\tclient:        &client,\n\t\t\trequest:       &mocks.ExecutePlanRequestSql,\n\t\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\t},\n\t}\n\n\t// Fetch the first response.\n\t_, err := stream.Recv()\n\tassert.NoError(t, err)\n\n\t_, err = stream.Recv()\n\tassert.NoError(t, err)\n}\n\nfunc Test_client_retriable_basics_execute(t *testing.T) {\n\tstream := mocks.NewProtoClientMock(&mocks.ExecutePlanResponseDone, &mocks.ExecutePlanResponseEOF)\n\tc := testutils.NewConnectServiceClientMock(stream, nil, nil, t)\n\tclient := retriableSparkConnectClient{\n\t\tclient:        c,\n\t\tsessionId:     mocks.MockSessionId,\n\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\toptions:       options.DefaultSparkClientOptions,\n\t}\n\tctx := context.Background()\n\tstream, err := client.ExecutePlan(ctx, &mocks.ExecutePlanRequestSql)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, stream)\n\n\t_, err = stream.Recv()\n\tassert.NoError(t, err)\n\n\t_, err = stream.Recv()\n\tassert.ErrorIs(t, err, io.EOF)\n}\n\nfunc Test_client_retriable_basics_analyze(t *testing.T) {\n\tc := testutils.NewConnectServiceClientMock(nil, mocks.AnalyzePlanResponse, nil, t)\n\tclient := retriableSparkConnectClient{\n\t\tclient:        c,\n\t\tsessionId:     mocks.MockSessionId,\n\t\tretryPolicies: []RetryPolicy{TestingRetryPolicy},\n\t\toptions:       options.DefaultSparkClientOptions,\n\t}\n\tctx := context.Background()\n\tresp, err := client.AnalyzePlan(ctx, &mocks.AnalyzePlanRequestSql)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, resp)\n\tassert.Equal(t, mocks.MockSessionId, resp.SessionId)\n}\n"
  },
  {
    "path": "spark/client/testutils/utils.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage testutils\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"google.golang.org/grpc\"\n)\n\n// connectServiceClient is a mock implementation of the SparkConnectServiceClient interface.\ntype connectServiceClient struct {\n\tt *testing.T\n\n\tanalysePlanResponse *proto.AnalyzePlanResponse\n\texecutePlanClient   proto.SparkConnectService_ExecutePlanClient\n\n\terr error\n}\n\n// FetchErrorDetails implements generated.SparkConnectServiceClient.\nfunc (c *connectServiceClient) FetchErrorDetails(ctx context.Context,\n\tin *proto.FetchErrorDetailsRequest, opts ...grpc.CallOption,\n) (*proto.FetchErrorDetailsResponse, error) {\n\tpanic(\"unimplemented\")\n}\n\n// ReleaseSession implements generated.SparkConnectServiceClient.\nfunc (c *connectServiceClient) ReleaseSession(ctx context.Context, in *proto.ReleaseSessionRequest,\n\topts ...grpc.CallOption,\n) (*proto.ReleaseSessionResponse, error) {\n\tpanic(\"unimplemented\")\n}\n\nfunc (c *connectServiceClient) ExecutePlan(ctx context.Context, in *proto.ExecutePlanRequest,\n\topts ...grpc.CallOption,\n) (proto.SparkConnectService_ExecutePlanClient, error) {\n\treturn c.executePlanClient, c.err\n}\n\nfunc (c *connectServiceClient) AnalyzePlan(ctx context.Context, in *proto.AnalyzePlanRequest,\n\topts ...grpc.CallOption,\n) (*proto.AnalyzePlanResponse, error) {\n\treturn c.analysePlanResponse, c.err\n}\n\nfunc (c *connectServiceClient) Config(ctx context.Context, in *proto.ConfigRequest, opts ...grpc.CallOption) (*proto.ConfigResponse, error) {\n\treturn nil, c.err\n}\n\nfunc (c *connectServiceClient) AddArtifacts(ctx context.Context, opts ...grpc.CallOption) (proto.SparkConnectService_AddArtifactsClient, error) {\n\treturn nil, c.err\n}\n\nfunc (c *connectServiceClient) ArtifactStatus(ctx context.Context,\n\tin *proto.ArtifactStatusesRequest, opts ...grpc.CallOption,\n) (*proto.ArtifactStatusesResponse, error) {\n\treturn nil, c.err\n}\n\nfunc (c *connectServiceClient) Interrupt(ctx context.Context, in *proto.InterruptRequest,\n\topts ...grpc.CallOption,\n) (*proto.InterruptResponse, error) {\n\treturn nil, c.err\n}\n\nfunc (c *connectServiceClient) ReattachExecute(ctx context.Context,\n\tin *proto.ReattachExecuteRequest, opts ...grpc.CallOption,\n) (proto.SparkConnectService_ReattachExecuteClient, error) {\n\treturn c.executePlanClient, c.err\n}\n\nfunc (c *connectServiceClient) ReleaseExecute(ctx context.Context, in *proto.ReleaseExecuteRequest,\n\topts ...grpc.CallOption,\n) (*proto.ReleaseExecuteResponse, error) {\n\treturn nil, c.err\n}\n\nfunc NewConnectServiceClientMock(epc proto.SparkConnectService_ExecutePlanClient,\n\tapr *proto.AnalyzePlanResponse, err error, t *testing.T,\n) proto.SparkConnectServiceClient {\n\treturn &connectServiceClient{\n\t\tt:                   t,\n\t\tanalysePlanResponse: apr,\n\t\texecutePlanClient:   epc,\n\t\terr:                 err,\n\t}\n}\n"
  },
  {
    "path": "spark/mocks/mock_executor.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage mocks\n\nimport (\n\t\"context\"\n\t\"errors\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/base\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n)\n\ntype TestExecutor struct {\n\tClient   base.ExecuteResponseStream\n\tresponse *generated.AnalyzePlanResponse\n\tErr      error\n}\n\nfunc (t *TestExecutor) ExecutePlan(ctx context.Context, plan *generated.Plan) (base.ExecuteResponseStream, error) {\n\tif t.Err != nil {\n\t\treturn nil, t.Err\n\t}\n\treturn t.Client, nil\n}\n\nfunc (t *TestExecutor) AnalyzePlan(ctx context.Context, plan *generated.Plan) (*generated.AnalyzePlanResponse, error) {\n\treturn t.response, nil\n}\n\nfunc (t *TestExecutor) Explain(ctx context.Context, plan *generated.Plan,\n\texplainMode utils.ExplainMode,\n) (*generated.AnalyzePlanResponse, error) {\n\treturn nil, errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) ExecuteCommand(ctx context.Context, plan *generated.Plan) (arrow.Table, *types.StructType, map[string]interface{}, error) {\n\tif t.Err != nil {\n\t\treturn nil, nil, nil, t.Err\n\t}\n\treturn nil, nil, nil, nil\n}\n\nfunc (t *TestExecutor) Persist(ctx context.Context, plan *generated.Plan, storageLevel utils.StorageLevel) error {\n\treturn errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) Unpersist(ctx context.Context, plan *generated.Plan) error {\n\treturn errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) GetStorageLevel(ctx context.Context, plan *generated.Plan) (*utils.StorageLevel, error) {\n\treturn nil, errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) SparkVersion(ctx context.Context) (string, error) {\n\treturn \"\", errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) DDLParse(ctx context.Context, sql string) (*types.StructType, error) {\n\treturn nil, errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) SameSemantics(ctx context.Context, plan1 *generated.Plan, plan2 *generated.Plan) (bool, error) {\n\treturn false, errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) SemanticHash(ctx context.Context, plan *generated.Plan) (int32, error) {\n\treturn 0, errors.New(\"not implemented\")\n}\n\nfunc (t *TestExecutor) Config(ctx context.Context, configRequest *generated.ConfigRequest_Operation) (*generated.ConfigResponse, error) {\n\treturn nil, errors.New(\"not implemented\")\n}\n"
  },
  {
    "path": "spark/mocks/mocks.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage mocks\n\nimport (\n\t\"context\"\n\t\"io\"\n\n\t\"google.golang.org/grpc/codes\"\n\t\"google.golang.org/grpc/status\"\n\n\t\"github.com/google/uuid\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"google.golang.org/grpc/metadata\"\n)\n\ntype MockResponse struct {\n\tResp *proto.ExecutePlanResponse\n\tErr  error\n}\n\ntype ProtoClient struct {\n\t// The stream of responses to return.\n\tRecvResponse []*MockResponse\n\tsent         int\n}\n\nvar (\n\tMockSessionId   = uuid.NewString()\n\tMockOperationId = uuid.NewString()\n\tMockResponseId  = \"1\"\n\tMockUserContext = &proto.UserContext{\n\t\tUserId: \"user\",\n\t}\n)\n\n// MockResponseDone is a response that indicates the plan execution is done.\nvar ExecutePlanResponseDone = MockResponse{\n\tResp: &proto.ExecutePlanResponse{\n\t\tResponseType: &proto.ExecutePlanResponse_ResultComplete_{\n\t\t\tResultComplete: &proto.ExecutePlanResponse_ResultComplete{},\n\t\t},\n\t\tSessionId:   MockSessionId,\n\t\tOperationId: MockOperationId,\n\t},\n\tErr: nil,\n}\n\nvar ExecutePlanResponseEOF = MockResponse{\n\tErr: io.EOF,\n}\n\nvar ExecutePlanResponseBrokenSchema = MockResponse{\n\tResp: &proto.ExecutePlanResponse{\n\t\tSchema: &proto.DataType{\n\t\t\tKind: &proto.DataType_String_{\n\t\t\t\tString_: &proto.DataType_String{},\n\t\t\t},\n\t\t},\n\t\tSessionId:   MockSessionId,\n\t\tOperationId: MockOperationId,\n\t},\n}\n\nvar ExecutePlanResponseWithSchema = MockResponse{\n\tResp: &proto.ExecutePlanResponse{\n\t\tResponseId:  MockResponseId,\n\t\tOperationId: MockOperationId,\n\t\tSchema: &proto.DataType{\n\t\t\tKind: &proto.DataType_Struct_{\n\t\t\t\tStruct: &proto.DataType_Struct{\n\t\t\t\t\tFields: []*proto.DataType_StructField{\n\t\t\t\t\t\t{\n\t\t\t\t\t\t\tName: \"col0\",\n\t\t\t\t\t\t\tDataType: &proto.DataType{\n\t\t\t\t\t\t\t\tKind: &proto.DataType_Integer_{\n\t\t\t\t\t\t\t\t\tInteger: &proto.DataType_Integer{},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\tNullable: true,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t},\n}\n\nvar ExecutePlanResponseUnavailable = MockResponse{\n\tErr: status.New(codes.Unavailable, \"Unavailable\").Err(),\n}\n\nvar ExecutePlanRequestSql = proto.ExecutePlanRequest{\n\tPlan:        NewSqlCommand(\"select range(10)\"),\n\tOperationId: &MockOperationId,\n\tSessionId:   MockSessionId,\n\tUserContext: MockUserContext,\n}\n\nvar AnalyzePlanRequestSql = proto.AnalyzePlanRequest{\n\tSessionId: MockSessionId,\n\tAnalyze: &proto.AnalyzePlanRequest_Schema_{\n\t\tSchema: &proto.AnalyzePlanRequest_Schema{\n\t\t\tPlan: NewSqlCommand(\"select range(10)\"),\n\t\t},\n\t},\n\tUserContext: MockUserContext,\n}\n\nvar AnalyzePlanResponse = &proto.AnalyzePlanResponse{\n\tSessionId: MockSessionId,\n\tResult: &proto.AnalyzePlanResponse_Schema_{\n\t\tSchema: &proto.AnalyzePlanResponse_Schema{\n\t\t\tSchema: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Struct_{\n\t\t\t\t\tStruct: &proto.DataType_Struct{\n\t\t\t\t\t\tFields: []*proto.DataType_StructField{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tName: \"col0\",\n\t\t\t\t\t\t\t\tDataType: &proto.DataType{\n\t\t\t\t\t\t\t\t\tKind: &proto.DataType_Integer_{\n\t\t\t\t\t\t\t\t\t\tInteger: &proto.DataType_Integer{},\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t},\n}\n\n// NewProtoClientMock creates a new mock client that returns the given responses.\nfunc NewProtoClientMock(responses ...*MockResponse) proto.SparkConnectService_ExecutePlanClient {\n\treturn &ProtoClient{RecvResponse: responses}\n}\n\nfunc (p *ProtoClient) Recv() (*proto.ExecutePlanResponse, error) {\n\tval := p.RecvResponse[p.sent]\n\tp.sent += 1\n\treturn val.Resp, val.Err\n}\n\nfunc (p *ProtoClient) Header() (metadata.MD, error) {\n\treturn nil, p.RecvResponse[p.sent].Err\n}\n\nfunc (p *ProtoClient) Trailer() metadata.MD {\n\treturn nil\n}\n\nfunc (p *ProtoClient) CloseSend() error {\n\treturn p.RecvResponse[p.sent].Err\n}\n\nfunc (p *ProtoClient) Context() context.Context {\n\treturn nil\n}\n\nfunc (p *ProtoClient) SendMsg(m interface{}) error {\n\treturn p.RecvResponse[p.sent].Err\n}\n\nfunc (p *ProtoClient) RecvMsg(m interface{}) error {\n\treturn p.RecvResponse[p.sent].Err\n}\n\nfunc NewSqlCommand(sql string) *proto.Plan {\n\treturn &proto.Plan{\n\t\tOpType: &proto.Plan_Command{\n\t\t\tCommand: &proto.Command{\n\t\t\t\tCommandType: &proto.Command_SqlCommand{\n\t\t\t\t\tSqlCommand: &proto.SqlCommand{\n\t\t\t\t\t\tSql: sql,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n}\n"
  },
  {
    "path": "spark/sparkerrors/errors.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sparkerrors\n\nimport (\n\t\"encoding/json\"\n\t\"fmt\"\n\t\"io\"\n\n\t\"github.com/go-errors/errors\"\n\t\"google.golang.org/genproto/googleapis/rpc/errdetails\"\n\t\"google.golang.org/grpc/codes\"\n\t\"google.golang.org/grpc/status\"\n)\n\ntype wrappedError struct {\n\terrorType error\n\tcause     *errors.Error\n}\n\nfunc (w *wrappedError) Unwrap() []error {\n\treturn []error{w.errorType, w.cause}\n}\n\nfunc (w *wrappedError) Error() string {\n\treturn fmt.Sprintf(\"%s\", w)\n}\n\n// WithType wraps an error with a type that can later be checked using `errors.Is`\nfunc WithType(err error, errType errorType) error {\n\treturn &wrappedError{cause: errors.Wrap(err, 1), errorType: errType}\n}\n\nfunc WithString(err error, errMsg string) error {\n\treturn &wrappedError{cause: errors.Wrap(err, 1), errorType: errors.New(errMsg)}\n}\n\nfunc WithStringf(err error, errMsg string, params ...any) error {\n\treturn &wrappedError{cause: errors.Wrap(err, 1), errorType: fmt.Errorf(errMsg, params...)}\n}\n\ntype errorType error\n\nvar (\n\tConnectionError               = errorType(errors.New(\"connection error\"))\n\tReadError                     = errorType(errors.New(\"read error\"))\n\tExecutionError                = errorType(errors.New(\"execution error\"))\n\tInvalidInputError             = errorType(errors.New(\"invalid input\"))\n\tInvalidPlanError              = errorType(errors.New(\"invalid plan\"))\n\tRetriesExceeded               = errorType(errors.New(\"retries exceeded\"))\n\tInvalidServerSideSessionError = errorType(errors.New(\"invalid server side session\"))\n\tTestSetupError                = errorType(errors.New(\"test setup error\"))\n\tWriteError                    = errorType(errors.New(\"write error\"))\n\tNotImplementedError           = errorType(errors.New(\"not implemented\"))\n\tInvalidArgumentError          = errorType(errors.New(\"invalid argument\"))\n)\n\n// Format formats the error, supporting both short forms (v, s, q) and verbose form (+v)\nfunc (w *wrappedError) Format(s fmt.State, verb rune) {\n\tswitch verb {\n\tcase 'v':\n\t\tif s.Flag('+') {\n\t\t\t_, _ = io.WriteString(s, \"[sparkerror] \")\n\t\t\t_, _ = io.WriteString(s, fmt.Sprintf(\"Error Type: %s\\n\", w.errorType.Error()))\n\t\t\t_, _ = io.WriteString(s, fmt.Sprintf(\"Error Cause: %s\\n%s\", w.cause.Err.Error(), w.cause.Stack()))\n\t\t\treturn\n\t\t}\n\t\tfallthrough\n\tcase 's':\n\t\t_, _ = io.WriteString(s, fmt.Sprintf(\"%s: %s\", w.errorType, w.cause))\n\tcase 'q':\n\t\t_, _ = fmt.Fprintf(s, \"%q\", w.errorType.Error())\n\t}\n}\n\ntype UnsupportedResponseTypeError struct {\n\tResponseType interface{}\n}\n\nfunc (e UnsupportedResponseTypeError) Error() string {\n\treturn fmt.Sprintf(\"Received unsupported response type: %T\", e.ResponseType)\n}\n\ntype InvalidServerSideSessionDetailsError struct {\n\tOwnSessionId      string\n\tReceivedSessionId string\n}\n\nfunc (e InvalidServerSideSessionDetailsError) Error() string {\n\treturn fmt.Sprintf(\"Received invalid session id %s, expected %s\", e.ReceivedSessionId, e.OwnSessionId)\n}\n\n// SparkError represents an error that is returned from Spark itself. It captures details of the\n// error that allows better understanding about the error. This allows us to check if the error\n// can be retried or not.\ntype SparkError struct {\n\t// SqlState is the SQL state of the error.\n\tSqlState string\n\t// ErrorClass is the class of the error.\n\tErrorClass string\n\t// If set is typically the classname throwing the error on the Spark side.\n\tReason string\n\t// Message is the human-readable message of the error.\n\tMessage string\n\t// Code is the gRPC status code of the error.\n\tCode codes.Code\n\t// ErrorId is the unique id of the error. It can be used to fetch more details about\n\t// the error using an additional RPC from the server.\n\tErrorId string\n\t// Parameters are the parameters that are used to format the error message.\n\tParameters map[string]string\n\tstatus     *status.Status\n}\n\nfunc (e SparkError) Error() string {\n\tif e.Code == codes.Internal && e.SqlState != \"\" {\n\t\treturn fmt.Sprintf(\"[%s] %s. SQLSTATE: %s\", e.ErrorClass, e.Message, e.SqlState)\n\t} else {\n\t\treturn fmt.Sprintf(\"[%s] %s\", e.Code.String(), e.Message)\n\t}\n}\n\n// FromRPCError converts a gRPC error to a SparkError. If the error is not a gRPC error, it will\n// create a plain \"UNKNOWN\" GRPC status type. If no error was observed returns nil.\nfunc FromRPCError(e error) *SparkError {\n\tstatus := status.Convert(e)\n\t// If there was no error, simply pass through.\n\tif status == nil {\n\t\treturn nil\n\t}\n\tresult := &SparkError{\n\t\tMessage: status.Message(),\n\t\tCode:    status.Code(),\n\t\tstatus:  status,\n\t}\n\n\t// Now lets, check if we can extract the error info from the details.\n\tfor _, d := range status.Details() {\n\t\tswitch info := d.(type) {\n\t\tcase *errdetails.ErrorInfo:\n\t\t\t// Parse the parameters from the error details, but only parse them if\n\t\t\t// they're present.\n\t\t\tvar params map[string]string\n\t\t\tif v, ok := info.GetMetadata()[\"messageParameters\"]; ok {\n\t\t\t\terr := json.Unmarshal([]byte(v), &params)\n\t\t\t\tif err == nil {\n\t\t\t\t\t// The message parameters is properly formatted JSON, if for some reason\n\t\t\t\t\t// this is not the case, errors are ignored.\n\t\t\t\t\tresult.Parameters = params\n\t\t\t\t}\n\t\t\t}\n\t\t\tresult.SqlState = info.GetMetadata()[\"sqlState\"]\n\t\t\tresult.ErrorClass = info.GetMetadata()[\"errorClass\"]\n\t\t\tresult.ErrorId = info.GetMetadata()[\"errorId\"]\n\t\t\tresult.Reason = info.Reason\n\t\t}\n\t}\n\treturn result\n}\n"
  },
  {
    "path": "spark/sparkerrors/errors_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sparkerrors\n\nimport (\n\t\"fmt\"\n\t\"testing\"\n\n\t\"google.golang.org/genproto/googleapis/rpc/errdetails\"\n\t\"google.golang.org/grpc/codes\"\n\t\"google.golang.org/grpc/status\"\n\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestWithTypeGivesAndErrorThatIsOfThatType(t *testing.T) {\n\terr := WithType(assert.AnError, ConnectionError)\n\tassert.ErrorIs(t, err, ConnectionError)\n}\n\nfunc TestErrorStringContainsErrorType(t *testing.T) {\n\terr := WithType(assert.AnError, ConnectionError)\n\tassert.Contains(t, err.Error(), ConnectionError.Error())\n}\n\nfunc TestGRPCErrorConversion(t *testing.T) {\n\terr := status.Error(codes.Internal, \"invalid argument\")\n\tse := FromRPCError(err)\n\tassert.Equal(t, se.Code, codes.Internal)\n\tassert.Equal(t, se.Message, \"invalid argument\")\n}\n\nfunc TestNonGRPCErrorsAreConvertedAsWell(t *testing.T) {\n\terr := assert.AnError\n\tse := FromRPCError(err)\n\tassert.Equal(t, se.Code, codes.Unknown)\n\tassert.Equal(t, se.Message, assert.AnError.Error())\n}\n\nfunc TestStackTracePrint(t *testing.T) {\n\terr := WithType(assert.AnError, ConnectionError)\n\terrorString := fmt.Sprintf(\"%+v\", err)\n\tt.Log(errorString)\n\tassert.Contains(t, errorString, \"spark/sparkerrors/errors_test.go\")\n}\n\nfunc TestErrorDetailsExtractionFromGRPCStatus(t *testing.T) {\n\tstatus := status.New(codes.Internal, \"AnalysisException\")\n\tstatus, _ = status.WithDetails(&errdetails.ErrorInfo{\n\t\tReason:   \"AnalysisException\",\n\t\tDomain:   \"spark.sql\",\n\t\tMetadata: map[string]string{},\n\t})\n\n\terr := status.Err()\n\tse := FromRPCError(err)\n\tassert.Equal(t, codes.Internal, se.Code)\n\tassert.Equal(t, \"AnalysisException\", se.Message)\n\tassert.Equal(t, \"AnalysisException\", se.Reason)\n}\n\nfunc TestErrorDetailsWithSqlStateAndClass(t *testing.T) {\n\tstatus := status.New(codes.Internal, \"AnalysisException\")\n\tstatus, _ = status.WithDetails(&errdetails.ErrorInfo{\n\t\tReason: \"AnalysisException\",\n\t\tDomain: \"spark.sql\",\n\t\tMetadata: map[string]string{\n\t\t\t\"sqlState\":          \"42000\",\n\t\t\t\"errorClass\":        \"ERROR_CLASS\",\n\t\t\t\"errorId\":           \"errorId\",\n\t\t\t\"messageParameters\": \"\",\n\t\t},\n\t})\n\n\terr := status.Err()\n\tse := FromRPCError(err)\n\tassert.Equal(t, codes.Internal, se.Code)\n\tassert.Equal(t, \"AnalysisException\", se.Message)\n\tassert.Equal(t, \"AnalysisException\", se.Reason)\n\tassert.Equal(t, \"42000\", se.SqlState)\n\tassert.Equal(t, \"ERROR_CLASS\", se.ErrorClass)\n\tassert.Equal(t, \"errorId\", se.ErrorId)\n}\n\nfunc TestErrorDetailsWithMessageParameterParsing(t *testing.T) {\n\ttype param struct {\n\t\tTestName string\n\t\tInput    string\n\t\tExpected map[string]string\n\t}\n\n\tparams := []param{\n\t\t{\"empty input\", \"\", nil},\n\t\t{\"empty input\", \"{\", nil},\n\t\t{\"parse error\", \"{}\", map[string]string{}},\n\t\t{\"valid input\", \"{\\\"key\\\":\\\"value\\\"}\", map[string]string{\"key\": \"value\"}},\n\t}\n\n\tfor _, p := range params {\n\t\tt.Run(p.TestName, func(t *testing.T) {\n\t\t\tstatus := status.New(codes.Internal, \"AnalysisException\")\n\t\t\tstatus, _ = status.WithDetails(&errdetails.ErrorInfo{\n\t\t\t\tReason: \"AnalysisException\",\n\t\t\t\tDomain: \"spark.sql\",\n\t\t\t\tMetadata: map[string]string{\n\t\t\t\t\t\"sqlState\":          \"42000\",\n\t\t\t\t\t\"errorClass\":        \"ERROR_CLASS\",\n\t\t\t\t\t\"errorId\":           \"errorId\",\n\t\t\t\t\t\"messageParameters\": p.Input,\n\t\t\t\t},\n\t\t\t})\n\n\t\t\terr := status.Err()\n\t\t\tse := FromRPCError(err)\n\t\t\tassert.Equal(t, codes.Internal, se.Code)\n\t\t\tassert.Equal(t, \"AnalysisException\", se.Message)\n\t\t\tassert.Equal(t, \"AnalysisException\", se.Reason)\n\t\t\tassert.Equal(t, \"42000\", se.SqlState)\n\t\t\tassert.Equal(t, \"ERROR_CLASS\", se.ErrorClass)\n\t\t\tassert.Equal(t, \"errorId\", se.ErrorId)\n\t\t\tassert.Equal(t, p.Expected, se.Parameters)\n\t\t})\n\t}\n}\n\nfunc TestSparkError_Error(t *testing.T) {\n\ttype fields struct {\n\t\tSqlState   string\n\t\tErrorClass string\n\t\tReason     string\n\t\tMessage    string\n\t\tCode       codes.Code\n\t\tErrorId    string\n\t\tParameters map[string]string\n\t\tstatus     *status.Status\n\t}\n\ttests := []struct {\n\t\tname   string\n\t\tfields fields\n\t\twant   string\n\t}{\n\t\t{\n\t\t\t\"UNKNOWN\",\n\t\t\tfields{\n\t\t\t\tCode:    codes.Unknown,\n\t\t\t\tMessage: \"Unknown error\",\n\t\t\t},\n\t\t\t\"[Unknown] Unknown error\",\n\t\t},\n\t\t{\n\t\t\t\"Analysis Exception\",\n\t\t\tfields{\n\t\t\t\tSqlState:   \"42703\",\n\t\t\t\tErrorClass: \"UNRESOLVED_COLUMN.WITH_SUGGESTION\",\n\t\t\t\tMessage:    \"A column, variable, or function parameter with name `id2` cannot be resolved. Did you mean one of the following? [`id`]\",\n\t\t\t\tCode:       codes.Internal,\n\t\t\t},\n\t\t\t\"[UNRESOLVED_COLUMN.WITH_SUGGESTION] A column, variable, or function parameter with name `id2` cannot be resolved. Did you mean one of the following? [`id`]. SQLSTATE: 42703\",\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\te := SparkError{\n\t\t\t\tSqlState:   tt.fields.SqlState,\n\t\t\t\tErrorClass: tt.fields.ErrorClass,\n\t\t\t\tReason:     tt.fields.Reason,\n\t\t\t\tMessage:    tt.fields.Message,\n\t\t\t\tCode:       tt.fields.Code,\n\t\t\t\tErrorId:    tt.fields.ErrorId,\n\t\t\t\tParameters: tt.fields.Parameters,\n\t\t\t\tstatus:     tt.fields.status,\n\t\t\t}\n\t\t\tassert.Equalf(t, tt.want, e.Error(), \"Error()\")\n\t\t})\n\t}\n}\n"
  },
  {
    "path": "spark/sql/column/column.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage column\n\nimport (\n\t\"context\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n)\n\n// Convertible is the interface for all things that can be converted into a protobuf expression.\ntype Convertible interface {\n\tToProto(ctx context.Context) (*proto.Expression, error)\n}\n\ntype Column struct {\n\texpr expression\n}\n\nfunc (c Column) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn c.expr.ToProto(ctx)\n}\n\nfunc (c Column) Lt(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\"<\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Le(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\"<=\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Gt(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\">\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Ge(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\">=\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Eq(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\"==\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Neq(other Column) Column {\n\tcmp := NewUnresolvedFunction(\"==\", []expression{c.expr, other.expr}, false)\n\treturn NewColumn(NewUnresolvedFunction(\"not\", []expression{cmp}, false))\n}\n\nfunc (c Column) Mul(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\"*\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Div(other Column) Column {\n\treturn NewColumn(NewUnresolvedFunction(\"/\", []expression{c.expr, other.expr}, false))\n}\n\nfunc (c Column) Desc() Column {\n\treturn NewColumn(&sortExpression{\n\t\tchild:        c.expr,\n\t\tdirection:    proto.Expression_SortOrder_SORT_DIRECTION_DESCENDING,\n\t\tnullOrdering: proto.Expression_SortOrder_SORT_NULLS_LAST,\n\t})\n}\n\nfunc (c Column) GetItem(key types.LiteralType) Column {\n\treturn NewColumn(NewUnresolvedExtractValue(\"getItem\", c.expr, NewLiteral(key)))\n}\n\nfunc (c Column) Asc() Column {\n\treturn NewColumn(&sortExpression{\n\t\tchild:        c.expr,\n\t\tdirection:    proto.Expression_SortOrder_SORT_DIRECTION_ASCENDING,\n\t\tnullOrdering: proto.Expression_SortOrder_SORT_NULLS_FIRST,\n\t})\n}\n\nfunc (c Column) Alias(alias string) Column {\n\treturn NewColumn(NewColumnAlias(alias, c.expr))\n}\n\nfunc NewColumn(expr expression) Column {\n\treturn Column{\n\t\texpr: expr,\n\t}\n}\n\ntype SchemaDataFrame interface {\n\tPlanId() int64\n\tSchema(ctx context.Context) (*types.StructType, error)\n}\n\nfunc OfDF(df SchemaDataFrame, colName string) Column {\n\treturn NewColumn(&delayedColumnReference{colName, df})\n}\n\nfunc OfDFWithRegex(df SchemaDataFrame, colRegex string) Column {\n\tplanId := df.PlanId()\n\treturn NewColumn(&unresolvedRegex{colRegex, &planId})\n}\n\ntype Alias struct {\n\tName string\n\tCol  Convertible\n}\n\nfunc (a Alias) ToProto(ctx context.Context) (*proto.Expression, error) {\n\tcol, err := a.Col.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Alias_{\n\t\t\tAlias: &proto.Expression_Alias{\n\t\t\t\tExpr: col,\n\t\t\t\tName: []string{a.Name},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc WithAlias(name string, col Convertible) Alias {\n\treturn Alias{\n\t\tName: name,\n\t\tCol:  col,\n\t}\n}\n"
  },
  {
    "path": "spark/sql/column/column_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage column\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestNewUnresolvedFunction_Basic(t *testing.T) {\n\tctx := context.Background()\n\tcol1 := NewColumn(NewColumnReference(\"col1\"))\n\tcol2 := NewColumn(NewColumnReference(\"col2\"))\n\tcol1Plan, _ := col1.ToProto(ctx)\n\tcol2Plan, _ := col2.ToProto(ctx)\n\n\ttype args struct {\n\t\tname       string\n\t\targuments  []expression\n\t\tisDistinct bool\n\t}\n\ttests := []struct {\n\t\tname string\n\t\targs args\n\t\twant *proto.Expression\n\t}{\n\t\t{\n\t\t\tname: \"TestNewUnresolvedWithArguments\",\n\t\t\targs: args{\n\t\t\t\tname:       \"id\",\n\t\t\t\targuments:  []expression{col1.expr, col2.expr},\n\t\t\t\tisDistinct: false,\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\tcol1Plan,\n\t\t\t\t\t\t\tcol2Plan,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNewUnresolvedWithArgumentsEmpty\",\n\t\t\targs: args{\n\t\t\t\tname:       \"id\",\n\t\t\t\targuments:  []expression{},\n\t\t\t\tisDistinct: true,\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tgot := NewUnresolvedFunction(tt.args.name, tt.args.arguments, tt.args.isDistinct)\n\t\t\texpected := tt.want\n\t\t\tp, err := got.ToProto(ctx)\n\t\t\tassert.NoError(t, err)\n\t\t\tassert.Equalf(t, expected, p, \"Input: %v\", tt.args)\n\t\t})\n\t}\n}\n\nfunc TestColumnFunctions(t *testing.T) {\n\tcol1 := NewColumn(NewColumnReference(\"col1\"))\n\tcol2 := NewColumn(NewColumnReference(\"col2\"))\n\n\tcol1Plan, _ := col1.ToProto(context.Background())\n\n\ttests := []struct {\n\t\tname string\n\t\targ  Column\n\t\twant *proto.Expression\n\t}{\n\t\t{\n\t\t\tname: \"TestColumnAlias\",\n\t\t\targ:  NewColumn(NewColumnAlias(\"alias\", col1.expr)),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_Alias_{\n\t\t\t\t\tAlias: &proto.Expression_Alias{\n\t\t\t\t\t\tExpr: col1Plan,\n\t\t\t\t\t\tName: []string{\"alias\"},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNewUnresolvedFunction\",\n\t\t\targ:  NewColumn(NewUnresolvedFunction(\"id\", nil, false)),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestLtComparison\",\n\t\t\targ:  col1.Lt(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"<\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestGtComparison\",\n\t\t\targ:  col1.Gt(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \">\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestLeComparison\",\n\t\t\targ:  col1.Le(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"<=\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestGeComparison\",\n\t\t\targ:  col1.Ge(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \">=\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestMulComparison\",\n\t\t\targ:  col1.Mul(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"*\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestDivComparison\",\n\t\t\targ:  col1.Div(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"/\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNeComparison\",\n\t\t\targ:  col1.Neq(col2),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"not\",\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\t\t\t\t\tFunctionName: \"==\",\n\t\t\t\t\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col1\",\n\t\t\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t\t{\n\t\t\t\t\t\t\t\t\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\t\t\t\t\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\t\t\t\t\t\t\t\t\tUnparsedIdentifier: \"col2\",\n\t\t\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tgot, err := tt.arg.ToProto(context.Background())\n\t\t\tassert.NoError(t, err)\n\t\t\texpected := tt.want\n\t\t\tassert.Equalf(t, expected, got, \"Input: %v\", tt.arg.expr.DebugString())\n\t\t})\n\t}\n}\n\nfunc TestColumn_Alias(t *testing.T) {\n\tcol1 := NewColumn(NewColumnReference(\"col1\"))\n\tcol1Plan, _ := col1.ToProto(context.Background())\n\n\ttests := []struct {\n\t\tname string\n\t\targ  Convertible\n\t\twant *proto.Expression\n\t}{\n\t\t{\n\t\t\tname: \"TestColumnAlias\",\n\t\t\targ:  WithAlias(\"alias\", col1),\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_Alias_{\n\t\t\t\t\tAlias: &proto.Expression_Alias{\n\t\t\t\t\t\tExpr: col1Plan,\n\t\t\t\t\t\tName: []string{\"alias\"},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tgot, err := tt.arg.ToProto(context.Background())\n\t\t\tassert.NoError(t, err)\n\t\t\texpected := tt.want\n\t\t\tassert.Equalf(t, expected, got, \"Input: %v\", tt.arg)\n\t\t})\n\t}\n}\n"
  },
  {
    "path": "spark/sql/column/expressions.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage column\n\nimport (\n\t\"context\"\n\t\"fmt\"\n\t\"strings\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n)\n\nfunc newProtoExpression() *proto.Expression {\n\treturn &proto.Expression{}\n}\n\n// expression is the interface for all expressions used by Spark Connect.\ntype expression interface {\n\tToProto(context.Context) (*proto.Expression, error)\n\tDebugString() string\n}\n\ntype unresolvedRegex struct {\n\tcolRegex string\n\tplanId   *int64\n}\n\nfunc (d *unresolvedRegex) DebugString() string {\n\treturn d.colRegex\n}\n\nfunc (d *unresolvedRegex) ToProto(ctx context.Context) (*proto.Expression, error) {\n\texpr := newProtoExpression()\n\texpr.ExprType = &proto.Expression_UnresolvedRegex_{\n\t\tUnresolvedRegex: &proto.Expression_UnresolvedRegex{\n\t\t\tColName: d.colRegex,\n\t\t\tPlanId:  d.planId,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype delayedColumnReference struct {\n\tunparsedIdentifier string\n\tdf                 SchemaDataFrame\n}\n\nfunc (d *delayedColumnReference) DebugString() string {\n\treturn d.unparsedIdentifier\n}\n\nfunc (d *delayedColumnReference) ToProto(ctx context.Context) (*proto.Expression, error) {\n\t// Check if the column identifier is actually part of the schema.\n\tschema, err := d.df.Schema(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tfound := false\n\tfor _, field := range schema.Fields {\n\t\tif field.Name == d.unparsedIdentifier {\n\t\t\tfound = true\n\t\t\tbreak\n\t\t}\n\t}\n\t// TODO: return proper pyspark error\n\tif !found {\n\t\treturn nil, sparkerrors.WithType(sparkerrors.InvalidPlanError,\n\t\t\tfmt.Errorf(\"cannot resolve column %s\", d.unparsedIdentifier))\n\t}\n\n\texpr := newProtoExpression()\n\tid := d.df.PlanId()\n\texpr.ExprType = &proto.Expression_UnresolvedAttribute_{\n\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\tUnparsedIdentifier: d.unparsedIdentifier,\n\t\t\tPlanId:             &id,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype sortExpression struct {\n\tchild        expression\n\tdirection    proto.Expression_SortOrder_SortDirection\n\tnullOrdering proto.Expression_SortOrder_NullOrdering\n}\n\nfunc (s *sortExpression) DebugString() string {\n\treturn s.child.DebugString()\n}\n\nfunc (s *sortExpression) ToProto(ctx context.Context) (*proto.Expression, error) {\n\texp := newProtoExpression()\n\tchild, err := s.child.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\texp.ExprType = &proto.Expression_SortOrder_{\n\t\tSortOrder: &proto.Expression_SortOrder{\n\t\t\tChild:        child,\n\t\t\tDirection:    s.direction,\n\t\t\tNullOrdering: s.nullOrdering,\n\t\t},\n\t}\n\treturn exp, nil\n}\n\ntype caseWhenExpression struct {\n\tbranches []*caseWhenBranch\n\telseExpr expression\n}\n\ntype caseWhenBranch struct {\n\tcondition expression\n\tvalue     expression\n}\n\nfunc NewCaseWhenExpression(branches []*caseWhenBranch, elseExpr expression) expression {\n\treturn &caseWhenExpression{branches: branches, elseExpr: elseExpr}\n}\n\nfunc (c *caseWhenExpression) DebugString() string {\n\tbranches := make([]string, 0)\n\tfor _, branch := range c.branches {\n\t\tbranches = append(branches, fmt.Sprintf(\"WHEN %s THEN %s\",\n\t\t\tbranch.condition.DebugString(), branch.value.DebugString()))\n\t}\n\n\telseExpr := \"\"\n\tif c.elseExpr != nil {\n\t\telseExpr = fmt.Sprintf(\"ELSE %s\", c.elseExpr.DebugString())\n\t}\n\n\treturn fmt.Sprintf(\"CASE %s %s END\", strings.Join(branches, \" \"), elseExpr)\n}\n\nfunc (c *caseWhenExpression) ToProto(ctx context.Context) (*proto.Expression, error) {\n\targs := make([]expression, 0)\n\tfor _, branch := range c.branches {\n\t\targs = append(args, branch.condition)\n\t\targs = append(args, branch.value)\n\t}\n\n\tif c.elseExpr != nil {\n\t\targs = append(args, c.elseExpr)\n\t}\n\n\tfun := NewUnresolvedFunction(\"when\", args, false)\n\treturn fun.ToProto(ctx)\n}\n\ntype unresolvedExtractValue struct {\n\tname       string\n\tchild      expression\n\textraction expression\n}\n\nfunc (u *unresolvedExtractValue) DebugString() string {\n\treturn fmt.Sprintf(\"%s(%s, %s)\", u.name, u.child.DebugString(), u.extraction.DebugString())\n}\n\nfunc (u *unresolvedExtractValue) ToProto(ctx context.Context) (*proto.Expression, error) {\n\texpr := newProtoExpression()\n\tchild, err := u.child.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\textraction, err := u.extraction.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\texpr.ExprType = &proto.Expression_UnresolvedExtractValue_{\n\t\tUnresolvedExtractValue: &proto.Expression_UnresolvedExtractValue{\n\t\t\tChild:      child,\n\t\t\tExtraction: extraction,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype unresolvedFunction struct {\n\tname       string\n\targs       []expression\n\tisDistinct bool\n}\n\nfunc (u *unresolvedFunction) DebugString() string {\n\targs := make([]string, 0)\n\tfor _, arg := range u.args {\n\t\targs = append(args, arg.DebugString())\n\t}\n\n\tdistinct := \"\"\n\tif u.isDistinct {\n\t\tdistinct = \"DISTINCT \"\n\t}\n\n\treturn fmt.Sprintf(\"%s(%s%s)\", u.name, distinct, strings.Join(args, \", \"))\n}\n\nfunc (u *unresolvedFunction) ToProto(ctx context.Context) (*proto.Expression, error) {\n\t// Convert input args to the proto expression.\n\tvar args []*proto.Expression = nil\n\tif len(u.args) > 0 {\n\t\targs = make([]*proto.Expression, 0)\n\t\tfor _, arg := range u.args {\n\t\t\tp, e := arg.ToProto(ctx)\n\t\t\tif e != nil {\n\t\t\t\treturn nil, e\n\t\t\t}\n\t\t\targs = append(args, p)\n\t\t}\n\t}\n\n\texpr := newProtoExpression()\n\texpr.ExprType = &proto.Expression_UnresolvedFunction_{\n\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\tFunctionName: u.name,\n\t\t\tArguments:    args,\n\t\t\tIsDistinct:   u.isDistinct,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\nfunc NewUnresolvedExtractValue(name string, child expression, extraction expression) expression {\n\treturn &unresolvedExtractValue{name: name, child: child, extraction: extraction}\n}\n\nfunc NewUnresolvedFunction(name string, args []expression, isDistinct bool) expression {\n\treturn &unresolvedFunction{name: name, args: args, isDistinct: isDistinct}\n}\n\nfunc NewUnresolvedFunctionWithColumns(name string, cols ...Column) expression {\n\texprs := make([]expression, 0)\n\tfor _, col := range cols {\n\t\texprs = append(exprs, col.expr)\n\t}\n\treturn NewUnresolvedFunction(name, exprs, false)\n}\n\ntype columnAlias struct {\n\talias    []string\n\texpr     expression\n\tmetadata *string\n}\n\nfunc NewColumnAlias(alias string, expr expression) expression {\n\treturn &columnAlias{alias: []string{alias}, expr: expr}\n}\n\nfunc NewColumnAliasFromNameParts(alias []string, expr expression) expression {\n\treturn &columnAlias{alias: alias, expr: expr}\n}\n\nfunc (c *columnAlias) DebugString() string {\n\tchild := c.expr.DebugString()\n\talias := strings.Join(c.alias, \".\")\n\treturn fmt.Sprintf(\"%s AS %s\", child, alias)\n}\n\nfunc (c *columnAlias) ToProto(ctx context.Context) (*proto.Expression, error) {\n\texpr := newProtoExpression()\n\talias, err := c.expr.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\texpr.ExprType = &proto.Expression_Alias_{\n\t\tAlias: &proto.Expression_Alias{\n\t\t\tExpr:     alias,\n\t\t\tName:     c.alias,\n\t\t\tMetadata: c.metadata,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype columnReference struct {\n\tunparsedIdentifier string\n\tplanId             *int64\n}\n\nfunc NewColumnReference(unparsedIdentifier string) expression {\n\treturn &columnReference{unparsedIdentifier: unparsedIdentifier}\n}\n\nfunc NewColumnReferenceWithPlanId(unparsedIdentifier string, planId int64) expression {\n\treturn &columnReference{unparsedIdentifier: unparsedIdentifier, planId: &planId}\n}\n\nfunc (c *columnReference) DebugString() string {\n\treturn c.unparsedIdentifier\n}\n\nfunc (c *columnReference) ToProto(context.Context) (*proto.Expression, error) {\n\texpr := newProtoExpression()\n\texpr.ExprType = &proto.Expression_UnresolvedAttribute_{\n\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\tUnparsedIdentifier: c.unparsedIdentifier,\n\t\t\tPlanId:             c.planId,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype sqlExression struct {\n\texpression_string string\n}\n\nfunc NewSQLExpression(expression string) expression {\n\treturn &sqlExression{expression_string: expression}\n}\n\nfunc (s *sqlExression) DebugString() string {\n\treturn s.expression_string\n}\n\nfunc (s *sqlExression) ToProto(context.Context) (*proto.Expression, error) {\n\texpr := newProtoExpression()\n\texpr.ExprType = &proto.Expression_ExpressionString_{\n\t\tExpressionString: &proto.Expression_ExpressionString{\n\t\t\tExpression: s.expression_string,\n\t\t},\n\t}\n\treturn expr, nil\n}\n\ntype literalExpression struct {\n\tvalue types.LiteralType\n}\n\nfunc (l *literalExpression) DebugString() string {\n\treturn fmt.Sprintf(\"%v\", l.value)\n}\n\nfunc (l *literalExpression) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn l.value.ToProto(ctx)\n}\n\nfunc NewLiteral(value types.LiteralType) expression {\n\treturn &literalExpression{value: value}\n}\n"
  },
  {
    "path": "spark/sql/column/expressions_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage column\n\nimport (\n\t\"context\"\n\t\"reflect\"\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestNewUnresolvedFunction(t *testing.T) {\n\tcolRef := NewColumnReference(\"martin\")\n\tcolRefPlan, _ := colRef.ToProto(context.Background())\n\ttype args struct {\n\t\tname       string\n\t\targuments  []expression\n\t\tisDistinct bool\n\t}\n\ttests := []struct {\n\t\tname string\n\t\targs args\n\t\twant *proto.Expression\n\t}{\n\t\t{\n\t\t\tname: \"TestNewUnresolvedFunction\",\n\t\t\targs: args{\n\t\t\t\tname:       \"id\",\n\t\t\t\targuments:  nil,\n\t\t\t\tisDistinct: false,\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNewUnresolvedWithArguments\",\n\t\t\targs: args{\n\t\t\t\tname:       \"id\",\n\t\t\t\targuments:  []expression{colRef},\n\t\t\t\tisDistinct: false,\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\tcolRefPlan,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tgot, err := NewUnresolvedFunction(tt.args.name, tt.args.arguments,\n\t\t\t\ttt.args.isDistinct).ToProto(context.Background())\n\t\t\tassert.NoError(t, err)\n\t\t\tif !reflect.DeepEqual(got, tt.want) {\n\t\t\t\tassert.Equal(t, tt.want, got)\n\t\t\t\tt.Errorf(\"NewUnresolvedFunction() = %v, want %v\", got, tt.want)\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestNewUnresolvedFunctionWithColumns(t *testing.T) {\n\tcolRef := NewColumn(NewColumnReference(\"martin\"))\n\tcolRefPlan, _ := colRef.ToProto(context.Background())\n\n\ttype args struct {\n\t\tname      string\n\t\targuments []Column\n\t}\n\ttests := []struct {\n\t\tname string\n\t\targs args\n\t\twant *proto.Expression\n\t}{\n\t\t{\n\t\t\tname: \"TestNewUnresolvedFunction\",\n\t\t\targs: args{\n\t\t\t\tname:      \"id\",\n\t\t\t\targuments: nil,\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNewUnresolvedWithArguments\",\n\t\t\targs: args{\n\t\t\t\tname:      \"id\",\n\t\t\t\targuments: []Column{colRef},\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\tcolRefPlan,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"TestNewUnresolvedWithManyArguments\",\n\t\t\targs: args{\n\t\t\t\tname:      \"id\",\n\t\t\t\targuments: []Column{colRef, colRef, colRef},\n\t\t\t},\n\t\t\twant: &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedFunction_{\n\t\t\t\t\tUnresolvedFunction: &proto.Expression_UnresolvedFunction{\n\t\t\t\t\t\tFunctionName: \"id\",\n\t\t\t\t\t\tIsDistinct:   false,\n\t\t\t\t\t\tArguments: []*proto.Expression{\n\t\t\t\t\t\t\tcolRefPlan,\n\t\t\t\t\t\t\tcolRefPlan,\n\t\t\t\t\t\t\tcolRefPlan,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tgot, err := NewUnresolvedFunctionWithColumns(tt.args.name,\n\t\t\t\ttt.args.arguments...).ToProto(context.Background())\n\t\t\tassert.NoError(t, err)\n\t\t\tif !reflect.DeepEqual(got, tt.want) {\n\t\t\t\tassert.Equal(t, tt.want, got)\n\t\t\t\tt.Errorf(\"NewUnresolvedFunction() = %v, want %v\", got, tt.want)\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestNewSQLExpression(t *testing.T) {\n\ttype args struct {\n\t\texpression string\n\t}\n\ttests := []struct {\n\t\tname string\n\t\targs args\n\t\twant *sqlExression\n\t}{\n\t\t{\n\t\t\tname: \"TestNewSQLExpression\",\n\t\t\targs: args{\n\t\t\t\texpression: \"id < 10\",\n\t\t\t},\n\t\t\twant: &sqlExression{\n\t\t\t\texpression_string: \"id < 10\",\n\t\t\t},\n\t\t},\n\t}\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tif got := NewSQLExpression(tt.args.expression); !reflect.DeepEqual(got, tt.want) {\n\t\t\t\tt.Errorf(\"NewSQLExpression() = %v, want %v\", got, tt.want)\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestColumnAlias_Basic(t *testing.T) {\n\tcolRef := NewColumnReference(\"column\")\n\tcolRefPlan, _ := colRef.ToProto(context.Background())\n\tcolAlias := NewColumnAlias(\"martin\", colRef)\n\tcolAliasPlan, _ := colAlias.ToProto(context.Background())\n\tassert.Equal(t, colRefPlan, colAliasPlan.GetAlias().GetExpr())\n\n\t// Test the debug string:\n\tassert.Equal(t, \"column AS martin\", colAlias.DebugString())\n}\n"
  },
  {
    "path": "spark/sql/dataframe.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\t\"fmt\"\n\t\"iter\"\n\t\"math/rand/v2\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/spark-connect-go/spark/sql/utils\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/column\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\n// ResultCollector receives a stream of result rows\ntype ResultCollector interface {\n\t// WriteRow receives a single row from the data frame\n\tWriteRow(values []any)\n}\n\n// DataFrame is a wrapper for data frame, representing a distributed collection of data row.\ntype DataFrame interface {\n\t// PlanId returns the plan id of the data frame.\n\tPlanId() int64\n\tAll(ctx context.Context) iter.Seq2[types.Row, error]\n\tAgg(ctx context.Context, exprs ...column.Convertible) (DataFrame, error)\n\tAggWithMap(ctx context.Context, exprs map[string]string) (DataFrame, error)\n\t// Alias creates a new DataFrame with the specified subquery alias\n\tAlias(ctx context.Context, alias string) DataFrame\n\tApproxQuantile(ctx context.Context, probabilities []float64, relativeError float64, cols ...string) ([][]float64, error)\n\t// Cache persists the DataFrame with the default storage level.\n\tCache(ctx context.Context) error\n\t// Coalesce returns a new DataFrame that has exactly numPartitions partitions.DataFrame\n\t//\n\t// Similar to coalesce defined on an :class:`RDD`, this operation results in a\n\t// narrow dependency, e.g. if you go from 1000 partitions to 100 partitions,\n\t// there will not be a shuffle, instead each of the 100 new partitions will\n\t// claim 10 of the current partitions. If a larger number of partitions is requested,\n\t// it will stay at the current number of partitions.\n\t//\n\t// However, if you're doing a drastic coalesce, e.g. to numPartitions = 1,\n\t// this may result in your computation taking place on fewer nodes than\n\t// you like (e.g. one node in the case of numPartitions = 1). To avoid this,\n\t// you can call repartition(). This will add a shuffle step, but means the\n\t// current upstream partitions will be executed in parallel (per whatever\n\t// the current partitioning is).\n\tCoalesce(ctx context.Context, numPartitions int) DataFrame\n\t// Columns returns the list of column names of the DataFrame.\n\tColumns(ctx context.Context) ([]string, error)\n\t// Corr calculates the correlation of two columns of a :class:`DataFrame` as a double value.\n\t// Currently only supports the Pearson Correlation Coefficient.\n\tCorr(ctx context.Context, col1, col2 string) (float64, error)\n\tCorrWithMethod(ctx context.Context, col1, col2 string, method string) (float64, error)\n\t// Count returns the number of rows in the DataFrame.\n\tCount(ctx context.Context) (int64, error)\n\t// Cov calculates the sample covariance for the given columns, specified by their names, as a\n\t// double value.\n\tCov(ctx context.Context, col1, col2 string) (float64, error)\n\t// Collect returns the data rows of the current data frame.\n\tCollect(ctx context.Context) ([]types.Row, error)\n\t// CreateTempView creates or replaces a temporary view.\n\tCreateTempView(ctx context.Context, viewName string, replace, global bool) error\n\t// CreateOrReplaceTempView creates or replaces a temporary view and replaces the optional existing view.\n\tCreateOrReplaceTempView(ctx context.Context, viewName string) error\n\t// CreateGlobalTempView creates a global temporary view.\n\tCreateGlobalTempView(ctx context.Context, viewName string) error\n\t// CreateOrReplaceGlobalTempView creates or replaces a global temporary view and replaces the optional existing view.\n\tCreateOrReplaceGlobalTempView(ctx context.Context, viewName string) error\n\t// CrossJoin joins the current DataFrame with another DataFrame using the cross product\n\tCrossJoin(ctx context.Context, other DataFrame) DataFrame\n\t// CrossTab computes a pair-wise frequency table of the given columns. Also known as a\n\t// contingency table.\n\t// The first column of each row will be the distinct values of `col1` and the column names\n\t// will be the distinct values of `col2`. The name of the first column will be `$col1_$col2`.\n\t// Pairs that have no occurrences will have zero as their counts.\n\tCrossTab(ctx context.Context, col1, col2 string) DataFrame\n\t// Cube creates a multi-dimensional cube for the current DataFrame using\n\t// the specified columns, so we can run aggregations on them.\n\tCube(ctx context.Context, cols ...column.Convertible) *GroupedData\n\t// Describe omputes basic statistics for numeric and string columns.\n\t// This includes count, mean, stddev, min, and max.\n\tDescribe(ctx context.Context, cols ...string) DataFrame\n\t// Distinct returns a new DataFrame containing the distinct rows in this DataFrame.\n\tDistinct(ctx context.Context) DataFrame\n\t// Drop returns a new DataFrame that drops the specified list of columns.\n\tDrop(ctx context.Context, columns ...column.Convertible) (DataFrame, error)\n\t// DropByName returns a new DataFrame that drops the specified list of columns by name.\n\tDropByName(ctx context.Context, columns ...string) (DataFrame, error)\n\t// DropDuplicates returns a new DataFrame that contains only the unique rows from this DataFrame.\n\tDropDuplicates(ctx context.Context, columns ...string) (DataFrame, error)\n\t// Drops all rows containing any null or NaN values. This is similar to PySparks dropna with how=any\n\tDropNa(ctx context.Context, cols ...string) (DataFrame, error)\n\t// Drops all rows containing all null or NaN values in the specified columns. This is\n\t// similar to PySparks dropna with how=all\n\tDropNaAll(ctx context.Context, cols ...string) (DataFrame, error)\n\t// Drops all rows containing null or NaN values in the specified columns. with a max threshold.\n\tDropNaWithThreshold(ctx context.Context, threshold int32, cols ...string) (DataFrame, error)\n\t// ExceptAll is similar to Substract but does not perform the distinct operation.\n\tExceptAll(ctx context.Context, other DataFrame) DataFrame\n\t// Explain returns the string explain plan for the current DataFrame according to the explainMode.\n\tExplain(ctx context.Context, explainMode utils.ExplainMode) (string, error)\n\t// FillNa replaces null values with specified value.\n\tFillNa(ctx context.Context, value types.PrimitiveTypeLiteral, columns ...string) (DataFrame, error)\n\t// FillNaWithValues replaces null values in specified columns (key of the map) with values.\n\tFillNaWithValues(ctx context.Context, values map[string]types.PrimitiveTypeLiteral) (DataFrame, error)\n\t// Filter filters the data frame by a column condition.\n\tFilter(ctx context.Context, condition column.Convertible) (DataFrame, error)\n\t// FilterByString filters the data frame by a string condition.\n\tFilterByString(ctx context.Context, condition string) (DataFrame, error)\n\t// Returns the first row of the DataFrame.\n\tFirst(ctx context.Context) (types.Row, error)\n\tFreqItems(ctx context.Context, cols ...string) DataFrame\n\tFreqItemsWithSupport(ctx context.Context, support float64, cols ...string) DataFrame\n\t// GetStorageLevel returns the storage level of the data frame.\n\tGetStorageLevel(ctx context.Context) (*utils.StorageLevel, error)\n\t// GroupBy groups the DataFrame by the spcified columns so that the aggregation\n\t// can be performed on them. See GroupedData for all the available aggregate functions.\n\tGroupBy(cols ...column.Convertible) *GroupedData\n\t// Head is an alias for Limit\n\tHead(ctx context.Context, limit int32) ([]types.Row, error)\n\t// Intersect performs the set intersection of two data frames and only returns distinct rows.\n\tIntersect(ctx context.Context, other DataFrame) DataFrame\n\t// IntersectAll performs the set intersection of two data frames and returns all rows.\n\tIntersectAll(ctx context.Context, other DataFrame) DataFrame\n\t// IsEmpty returns true if the DataFrame is empty.\n\tIsEmpty(ctx context.Context) (bool, error)\n\t// Join joins the current DataFrame with another DataFrame using the specified column using the joinType specified.\n\tJoin(ctx context.Context, other DataFrame, on column.Convertible, joinType utils.JoinType) (DataFrame, error)\n\t// Limit applies a limit on the DataFrame\n\tLimit(ctx context.Context, limit int32) DataFrame\n\t// Melt is an alias for Unpivot.\n\tMelt(ctx context.Context, ids []column.Convertible, values []column.Convertible,\n\t\tvariableColumnName string, valueColumnName string) (DataFrame, error)\n\tNa() DataFrameNaFunctions\n\t// Offset returns a new DataFrame by skipping the first `offset` rows.\n\tOffset(ctx context.Context, offset int32) DataFrame\n\t// OrderBy is an alias for Sort\n\tOrderBy(ctx context.Context, columns ...column.Convertible) (DataFrame, error)\n\tPrintSchema(ctx context.Context) error\n\tPersist(ctx context.Context, storageLevel utils.StorageLevel) error\n\tRandomSplit(ctx context.Context, weights []float64) ([]DataFrame, error)\n\t// Repartition re-partitions a data frame.\n\tRepartition(ctx context.Context, numPartitions int, columns []string) (DataFrame, error)\n\t// RepartitionByRange re-partitions a data frame by range partition.\n\tRepartitionByRange(ctx context.Context, numPartitions int, columns ...column.Convertible) (DataFrame, error)\n\t// Replace Returns a new DataFrame` replacing a value with another value.\n\t// Values toReplace and Values must have the same type and can only be numerics, booleans,\n\t// or strings. Value can have None. When replacing, the new value will be cast\n\t// to the type of the existing column.\n\t//\n\t// For numeric replacements all values to be replaced should have unique\n\t// floating point representation. If cols is set allows to specify a subset of columns to\n\t// perform the replacement.\n\tReplace(ctx context.Context, toReplace []types.PrimitiveTypeLiteral,\n\t\tvalues []types.PrimitiveTypeLiteral, cols ...string) (DataFrame, error)\n\t// Rollup creates a multi-dimensional rollup for the current DataFrame using\n\t// the specified columns, so we can run aggregation on them.\n\tRollup(ctx context.Context, cols ...column.Convertible) *GroupedData\n\t// SameSemantics returns true if the other DataFrame has the same semantics.\n\tSameSemantics(ctx context.Context, other DataFrame) (bool, error)\n\t// Sample samples a data frame without replacement and random seed.\n\tSample(ctx context.Context, fraction float64) (DataFrame, error)\n\t// SampleWithReplacement samples a data frame with random seed and with/without replacement.\n\tSampleWithReplacement(ctx context.Context, withReplacement bool, fraction float64) (DataFrame, error)\n\t// SampleWithSeed samples a data frame without replacement and given seed.\n\tSampleWithSeed(ctx context.Context, fraction float64, seed int64) (DataFrame, error)\n\t// SampleWithReplacementAndSeed samples a data frame with/without replacement and given seed.\n\tSampleWithReplacementAndSeed(ctx context.Context, withReplacement bool, fraction float64, seed int64) (DataFrame, error)\n\t// Show uses WriteResult to write the data frames to the console output.\n\tShow(ctx context.Context, numRows int, truncate bool) error\n\t// Schema returns the schema for the current data frame.\n\tSchema(ctx context.Context) (*types.StructType, error)\n\t// Select projects a list of columns from the DataFrame\n\tSelect(ctx context.Context, columns ...column.Convertible) (DataFrame, error)\n\t// SelectExpr projects a list of columns from the DataFrame by string expressions\n\tSelectExpr(ctx context.Context, exprs ...string) (DataFrame, error)\n\t// SemanticHash returns the semantic hash of the data frame. The semantic hash can be used to\n\t// understand of the semantic operations are similar.\n\tSemanticHash(ctx context.Context) (int32, error)\n\t// Sort returns a new DataFrame sorted by the specified columns.\n\tSort(ctx context.Context, columns ...column.Convertible) (DataFrame, error)\n\tStat() DataFrameStatFunctions\n\t// Subtract subtracts the other DataFrame from the current DataFrame. And only returns\n\t// distinct rows.\n\tSubtract(ctx context.Context, other DataFrame) DataFrame\n\t// Summary computes the specified statistics for the current DataFrame and returns it\n\t// as a new DataFrame. Available statistics are: \"count\", \"mean\", \"stddev\", \"min\", \"max\" and\n\t// arbitrary percentiles specified as a percentage (e.g., \"75%\"). If no statistics are given,\n\t// this function computes \"count\", \"mean\", \"stddev\", \"min\", \"25%\", \"50%\", \"75%\", \"max\".\n\tSummary(ctx context.Context, statistics ...string) DataFrame\n\t// Tail returns the last `limit` rows as a list of Row.\n\tTail(ctx context.Context, limit int32) ([]types.Row, error)\n\t// Take is an alias for Limit\n\tTake(ctx context.Context, limit int32) ([]types.Row, error)\n\t// ToArrow returns the Arrow representation of the DataFrame.\n\tToArrow(ctx context.Context) (*arrow.Table, error)\n\t// Union is an alias for UnionAll\n\tUnion(ctx context.Context, other DataFrame) DataFrame\n\t// UnionAll returns a new DataFrame containing union of rows in this and another DataFrame.\n\tUnionAll(ctx context.Context, other DataFrame) DataFrame\n\t// UnionByName performs a SQL union operation on two dataframes but reorders the schema\n\t// according to the matching columns. If columns are missing, it will throw an eror.\n\tUnionByName(ctx context.Context, other DataFrame) DataFrame\n\t// UnionByNameWithMissingColumns performs a SQL union operation on two dataframes but reorders the schema\n\t// according to the matching columns. Missing columns are supported.\n\tUnionByNameWithMissingColumns(ctx context.Context, other DataFrame) DataFrame\n\t// Unpersist resets the storage level for this data frame, and if necessary removes it\n\t// from server-side caches.\n\tUnpersist(ctx context.Context) error\n\t// Unpivot a DataFrame from wide format to long format, optionally leaving\n\t// identifier columns set. This is the reverse to `groupBy(...).pivot(...).agg(...)`,\n\t// except for the aggregation, which cannot be reversed.\n\t//\n\t// This function is useful to massage a DataFrame into a format where some\n\t// columns are identifier columns (\"ids\"), while all other columns (\"values\")\n\t// are \"unpivoted\" to the rows, leaving just two non-id columns, named as given\n\t// by `variableColumnName` and `valueColumnName`.\n\t//\n\t// When no \"id\" columns are given, the unpivoted DataFrame consists of only the\n\t// \"variable\" and \"value\" columns.\n\t//\n\t// The `values` columns must not be empty so at least one value must be given to be unpivoted.\n\t// When `values` is `None`, all non-id columns will be unpivoted.\n\t//\n\t// All \"value\" columns must share a least common data type. Unless they are the same data type,\n\t// all \"value\" columns are cast to the nearest common data type. For instance, types\n\t// `IntegerType` and `LongType` are cast to `LongType`, while `IntegerType` and `StringType`\n\t// do not have a common data type and `unpivot` fails.\n\tUnpivot(ctx context.Context, ids []column.Convertible, values []column.Convertible,\n\t\tvariableColumnName string, valueColumnName string) (DataFrame, error)\n\t// WithColumn returns a new DataFrame by adding a column or replacing the\n\t// existing column that has the same name. The column expression must be an\n\t// expression over this DataFrame; attempting to add a column from some other\n\t// DataFrame will raise an error.\n\t//\n\t// Note: This method introduces a projection internally. Therefore, calling it multiple\n\t// times, for instance, via loops in order to add multiple columns can generate big\n\t// plans which can cause performance issues and even `StackOverflowException`.\n\t// To avoid this, use :func:`select` with multiple columns at once.\n\tWithColumn(ctx context.Context, colName string, col column.Convertible) (DataFrame, error)\n\tWithColumns(ctx context.Context, alias ...column.Alias) (DataFrame, error)\n\t// WithColumnRenamed returns a new DataFrame by renaming an existing column.\n\t// This is a no-op if the schema doesn't contain the given column name.\n\tWithColumnRenamed(ctx context.Context, existingName, newName string) (DataFrame, error)\n\t// WithColumnsRenamed returns a new DataFrame by renaming multiple existing columns.\n\tWithColumnsRenamed(ctx context.Context, colsMap map[string]string) (DataFrame, error)\n\t// WithMetadata returns a new DataFrame with the specified metadata for each of the columns.\n\tWithMetadata(ctx context.Context, metadata map[string]string) (DataFrame, error)\n\tWithWatermark(ctx context.Context, eventTime string, delayThreshold string) (DataFrame, error)\n\tWhere(ctx context.Context, condition string) (DataFrame, error)\n\t// Writer returns a data frame writer, which could be used to save data frame to supported storage.\n\tWriter() DataFrameWriter\n\t// Write is an alias for Writer\n\t// Deprecated: Use Writer\n\tWrite() DataFrameWriter\n\t// WriteResult streams the data frames to a result collector\n\tWriteResult(ctx context.Context, collector ResultCollector, numRows int, truncate bool) error\n}\n\n// dataFrameImpl is an implementation of DataFrame interface.\ntype dataFrameImpl struct {\n\tsession  *sparkSessionImpl\n\trelation *proto.Relation // TODO change to proto.Plan?\n}\n\nfunc (df *dataFrameImpl) Coalesce(ctx context.Context, numPartitions int) DataFrame {\n\tshuffle := false\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Repartition{\n\t\t\tRepartition: &proto.Repartition{\n\t\t\t\tInput:         df.relation,\n\t\t\t\tShuffle:       &shuffle,\n\t\t\t\tNumPartitions: int32(numPartitions),\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Columns(ctx context.Context) ([]string, error) {\n\tschema, err := df.Schema(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tcolumns := make([]string, len(schema.Fields))\n\tfor i, field := range schema.Fields {\n\t\tcolumns[i] = field.Name\n\t}\n\treturn columns, nil\n}\n\nfunc (df *dataFrameImpl) Corr(ctx context.Context, col1, col2 string) (float64, error) {\n\treturn df.CorrWithMethod(ctx, col1, col2, \"pearson\")\n}\n\nfunc (df *dataFrameImpl) CorrWithMethod(ctx context.Context, col1, col2 string, method string) (float64, error) {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: &proto.Relation{\n\t\t\t\tCommon: &proto.RelationCommon{\n\t\t\t\t\tPlanId: newPlanId(),\n\t\t\t\t},\n\t\t\t\tRelType: &proto.Relation_Corr{\n\t\t\t\t\tCorr: &proto.StatCorr{\n\t\t\t\t\t\tInput:  df.relation,\n\t\t\t\t\t\tCol1:   col1,\n\t\t\t\t\t\tCol2:   col2,\n\t\t\t\t\t\tMethod: &method,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, plan)\n\tif err != nil {\n\t\treturn 0, sparkerrors.WithType(fmt.Errorf(\"failed to execute plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\t_, table, err := responseClient.ToTable()\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\n\tvalues, err := types.ReadArrowTableToRows(table)\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\n\treturn values[0].At(0).(float64), nil\n}\n\nfunc (df *dataFrameImpl) Count(ctx context.Context) (int64, error) {\n\tres, err := df.GroupBy().Count(ctx)\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\trows, err := res.Collect(ctx)\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\n\treturn rows[0].At(0).(int64), nil\n}\n\nfunc (df *dataFrameImpl) Cov(ctx context.Context, col1, col2 string) (float64, error) {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: &proto.Relation{\n\t\t\t\tCommon: &proto.RelationCommon{\n\t\t\t\t\tPlanId: newPlanId(),\n\t\t\t\t},\n\t\t\t\tRelType: &proto.Relation_Cov{\n\t\t\t\t\tCov: &proto.StatCov{\n\t\t\t\t\t\tInput: df.relation,\n\t\t\t\t\t\tCol1:  col1,\n\t\t\t\t\t\tCol2:  col2,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, plan)\n\tif err != nil {\n\t\treturn 0, sparkerrors.WithType(fmt.Errorf(\"failed to execute plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\t_, table, err := responseClient.ToTable()\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\n\tvalues, err := types.ReadArrowTableToRows(table)\n\tif err != nil {\n\t\treturn 0, err\n\t}\n\n\treturn values[0].At(0).(float64), nil\n}\n\nfunc (df *dataFrameImpl) PlanId() int64 {\n\treturn df.relation.GetCommon().GetPlanId()\n}\n\nfunc (df *dataFrameImpl) SelectExpr(ctx context.Context, exprs ...string) (DataFrame, error) {\n\texpressions := make([]*proto.Expression, 0, len(exprs))\n\tfor _, expr := range exprs {\n\t\tcol := column.NewSQLExpression(expr)\n\t\tf, e := col.ToProto(ctx)\n\t\tif e != nil {\n\t\t\treturn nil, e\n\t\t}\n\t\texpressions = append(expressions, f)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Project{\n\t\t\tProject: &proto.Project{\n\t\t\t\tInput:       df.relation,\n\t\t\t\tExpressions: expressions,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Alias(ctx context.Context, alias string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SubqueryAlias{\n\t\t\tSubqueryAlias: &proto.SubqueryAlias{\n\t\t\t\tInput: df.relation,\n\t\t\t\tAlias: alias,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) CrossJoin(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Join{\n\t\t\tJoin: &proto.Join{\n\t\t\t\tLeft:     df.relation,\n\t\t\t\tRight:    otherDf.relation,\n\t\t\t\tJoinType: proto.Join_JOIN_TYPE_CROSS,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\n// NewDataFrame creates a new DataFrame\nfunc NewDataFrame(session *sparkSessionImpl, relation *proto.Relation) DataFrame {\n\treturn &dataFrameImpl{\n\t\tsession:  session,\n\t\trelation: relation,\n\t}\n}\n\ntype consoleCollector struct{}\n\nfunc (c consoleCollector) WriteRow(values []any) {\n\tfmt.Println(values...)\n}\n\nfunc (df *dataFrameImpl) Show(ctx context.Context, numRows int, truncate bool) error {\n\treturn df.WriteResult(ctx, &consoleCollector{}, numRows, truncate)\n}\n\nfunc (df *dataFrameImpl) WriteResult(ctx context.Context, collector ResultCollector, numRows int, truncate bool) error {\n\ttruncateValue := 0\n\tif truncate {\n\t\ttruncateValue = 20\n\t}\n\tvertical := false\n\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: &proto.Relation{\n\t\t\t\tCommon: &proto.RelationCommon{\n\t\t\t\t\tPlanId: newPlanId(),\n\t\t\t\t},\n\t\t\t\tRelType: &proto.Relation_ShowString{\n\t\t\t\t\tShowString: &proto.ShowString{\n\t\t\t\t\t\tInput:    df.relation,\n\t\t\t\t\t\tNumRows:  int32(numRows),\n\t\t\t\t\t\tTruncate: int32(truncateValue),\n\t\t\t\t\t\tVertical: vertical,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, plan)\n\tif err != nil {\n\t\treturn sparkerrors.WithType(fmt.Errorf(\"failed to show dataframe: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\t_, table, err := responseClient.ToTable()\n\tif err != nil {\n\t\treturn err\n\t}\n\n\trows, err := types.ReadArrowTableToRows(table)\n\tif err != nil {\n\t\treturn err\n\t}\n\n\tfor _, row := range rows {\n\t\tvalues := row.Values()\n\t\tcollector.WriteRow(values)\n\t}\n\treturn nil\n}\n\nfunc (df *dataFrameImpl) Schema(ctx context.Context) (*types.StructType, error) {\n\tresponse, err := df.session.client.AnalyzePlan(ctx, df.createPlan())\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to analyze plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\tresponseSchema := response.GetSchema().Schema\n\treturn types.ConvertProtoDataTypeToStructType(responseSchema)\n}\n\nfunc (df *dataFrameImpl) Collect(ctx context.Context) ([]types.Row, error) {\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, df.createPlan())\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to execute plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\t_, table, err := responseClient.ToTable()\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn types.ReadArrowTableToRows(table)\n}\n\nfunc (df *dataFrameImpl) Write() DataFrameWriter {\n\treturn df.Writer()\n}\n\nfunc (df *dataFrameImpl) Writer() DataFrameWriter {\n\treturn newDataFrameWriter(df.session, df.relation)\n}\n\nfunc (df *dataFrameImpl) CreateTempView(ctx context.Context, viewName string, replace, global bool) error {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Command{\n\t\t\tCommand: &proto.Command{\n\t\t\t\tCommandType: &proto.Command_CreateDataframeView{\n\t\t\t\t\tCreateDataframeView: &proto.CreateDataFrameViewCommand{\n\t\t\t\t\t\tInput:    df.relation,\n\t\t\t\t\t\tName:     viewName,\n\t\t\t\t\t\tReplace:  replace,\n\t\t\t\t\t\tIsGlobal: global,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, plan)\n\tif err != nil {\n\t\treturn sparkerrors.WithType(fmt.Errorf(\"failed to create temp view %s: %w\",\n\t\t\tviewName, err), sparkerrors.ExecutionError)\n\t}\n\n\t_, _, err = responseClient.ToTable()\n\treturn err\n}\n\nfunc (df *dataFrameImpl) CreateOrReplaceTempView(ctx context.Context, viewName string) error {\n\treturn df.CreateTempView(ctx, viewName, true, false)\n}\n\nfunc (df *dataFrameImpl) CreateGlobalTempView(ctx context.Context, viewName string) error {\n\treturn df.CreateTempView(ctx, viewName, false, true)\n}\n\nfunc (df *dataFrameImpl) CreateOrReplaceGlobalTempView(ctx context.Context, viewName string) error {\n\treturn df.CreateTempView(ctx, viewName, true, true)\n}\n\nfunc (df *dataFrameImpl) Repartition(ctx context.Context, numPartitions int, columns []string) (DataFrame, error) {\n\tvar partitionExpressions []*proto.Expression\n\tif columns != nil {\n\t\tpartitionExpressions = make([]*proto.Expression, 0, len(columns))\n\t\tfor _, c := range columns {\n\t\t\texpr := &proto.Expression{\n\t\t\t\tExprType: &proto.Expression_UnresolvedAttribute_{\n\t\t\t\t\tUnresolvedAttribute: &proto.Expression_UnresolvedAttribute{\n\t\t\t\t\t\tUnparsedIdentifier: c,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t}\n\t\t\tpartitionExpressions = append(partitionExpressions, expr)\n\t\t}\n\t}\n\treturn df.repartitionByExpressions(numPartitions, partitionExpressions)\n}\n\nfunc (df *dataFrameImpl) RepartitionByRange(ctx context.Context, numPartitions int, columns ...column.Convertible) (DataFrame, error) {\n\tvar partitionExpressions []*proto.Expression\n\tif columns != nil {\n\t\tpartitionExpressions = make([]*proto.Expression, 0, len(columns))\n\t\tfor _, c := range columns {\n\t\t\texpr, err := c.ToProto(ctx)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, err\n\t\t\t}\n\t\t\tpartitionExpressions = append(partitionExpressions, expr)\n\t\t}\n\t}\n\treturn df.repartitionByExpressions(numPartitions, partitionExpressions)\n}\n\nfunc (df *dataFrameImpl) createPlan() *proto.Plan {\n\treturn &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n}\n\nfunc (df *dataFrameImpl) repartitionByExpressions(numPartitions int,\n\tpartitionExpressions []*proto.Expression,\n) (DataFrame, error) {\n\tvar numPartitionsPointerValue *int32\n\tif numPartitions != 0 {\n\t\tint32Value := int32(numPartitions)\n\t\tnumPartitionsPointerValue = &int32Value\n\t}\n\tdf.relation.GetRepartitionByExpression()\n\tnewRelation := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_RepartitionByExpression{\n\t\t\tRepartitionByExpression: &proto.RepartitionByExpression{\n\t\t\t\tInput:          df.relation,\n\t\t\t\tNumPartitions:  numPartitionsPointerValue,\n\t\t\t\tPartitionExprs: partitionExpressions,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, newRelation), nil\n}\n\nfunc (df *dataFrameImpl) Filter(ctx context.Context, condition column.Convertible) (DataFrame, error) {\n\tcnd, err := condition.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Filter{\n\t\t\tFilter: &proto.Filter{\n\t\t\t\tInput:     df.relation,\n\t\t\t\tCondition: cnd,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) FilterByString(ctx context.Context, condition string) (DataFrame, error) {\n\treturn df.Filter(ctx, column.NewColumn(column.NewSQLExpression(condition)))\n}\n\nfunc (df *dataFrameImpl) Select(ctx context.Context, columns ...column.Convertible) (DataFrame, error) {\n\t//\n\tif len(columns) == 0 {\n\t\treturn df, nil\n\t}\n\texprs := make([]*proto.Expression, 0, len(columns))\n\tfor _, c := range columns {\n\t\texpr, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\texprs = append(exprs, expr)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Project{\n\t\t\tProject: &proto.Project{\n\t\t\t\tInput:       df.relation,\n\t\t\t\tExpressions: exprs,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\n// GroupBy groups the DataFrame by the specified columns so that aggregation\n// can be performed on them. See GroupedData for all the available aggregate functions.\nfunc (df *dataFrameImpl) GroupBy(cols ...column.Convertible) *GroupedData {\n\treturn &GroupedData{\n\t\tdf:           df,\n\t\tgroupingCols: cols,\n\t\tgroupType:    \"groupby\",\n\t}\n}\n\nfunc (df *dataFrameImpl) WithColumn(ctx context.Context, colName string, col column.Convertible) (DataFrame, error) {\n\treturn df.WithColumns(ctx, column.WithAlias(colName, col))\n}\n\nfunc (df *dataFrameImpl) WithColumns(ctx context.Context, cols ...column.Alias) (DataFrame, error) {\n\t// Convert all columns to proto expressions and the corresponding alias:\n\taliases := make([]*proto.Expression_Alias, 0, len(cols))\n\tfor _, col := range cols {\n\t\texpr, err := col.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\t// The alias must be an alias expression.\n\t\talias := expr.GetAlias()\n\t\taliases = append(aliases, alias)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_WithColumns{\n\t\t\tWithColumns: &proto.WithColumns{\n\t\t\t\tInput:   df.relation,\n\t\t\t\tAliases: aliases,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) WithColumnRenamed(ctx context.Context, existingName, newName string) (DataFrame, error) {\n\treturn df.WithColumnsRenamed(ctx, map[string]string{existingName: newName})\n}\n\nfunc (df *dataFrameImpl) WithColumnsRenamed(ctx context.Context, colsMap map[string]string) (DataFrame, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_WithColumnsRenamed{\n\t\t\tWithColumnsRenamed: &proto.WithColumnsRenamed{\n\t\t\t\tInput:            df.relation,\n\t\t\t\tRenameColumnsMap: colsMap,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) WithMetadata(ctx context.Context, metadata map[string]string) (DataFrame, error) {\n\t// WithMetadata works the same way as with columns but extracts the column reference from the DataFrame\n\t// and injects it back into the projection.\n\taliases := make([]*proto.Expression_Alias, 0, len(metadata))\n\tfor colName, metadata := range metadata {\n\t\texpr := column.OfDF(df, colName)\n\t\texprProto, err := expr.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\talias := &proto.Expression_Alias{\n\t\t\tExpr:     exprProto,\n\t\t\tName:     []string{colName},\n\t\t\tMetadata: &metadata,\n\t\t}\n\t\taliases = append(aliases, alias)\n\t}\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_WithColumns{\n\t\t\tWithColumns: &proto.WithColumns{\n\t\t\t\tInput:   df.relation,\n\t\t\t\tAliases: aliases,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) WithWatermark(ctx context.Context, eventTime string, delayThreshold string) (DataFrame, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_WithWatermark{\n\t\t\tWithWatermark: &proto.WithWatermark{\n\t\t\t\tInput:          df.relation,\n\t\t\t\tEventTime:      eventTime,\n\t\t\t\tDelayThreshold: delayThreshold,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Where(ctx context.Context, condition string) (DataFrame, error) {\n\treturn df.FilterByString(ctx, condition)\n}\n\nfunc (df *dataFrameImpl) Drop(ctx context.Context, columns ...column.Convertible) (DataFrame, error) {\n\texprs := make([]*proto.Expression, 0, len(columns))\n\tfor _, c := range columns {\n\t\te, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\texprs = append(exprs, e)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Drop{\n\t\t\tDrop: &proto.Drop{\n\t\t\t\tInput:   df.relation,\n\t\t\t\tColumns: exprs,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) DropByName(ctx context.Context, columns ...string) (DataFrame, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Drop{\n\t\t\tDrop: &proto.Drop{\n\t\t\t\tInput:       df.relation,\n\t\t\t\tColumnNames: columns,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) DropDuplicates(ctx context.Context, columns ...string) (DataFrame, error) {\n\twithinWatermark := false\n\tallColumnsAsKeys := len(columns) == 0\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Deduplicate{\n\t\t\tDeduplicate: &proto.Deduplicate{\n\t\t\t\tInput:            df.relation,\n\t\t\t\tColumnNames:      columns,\n\t\t\t\tWithinWatermark:  &withinWatermark,\n\t\t\t\tAllColumnsAsKeys: &allColumnsAsKeys,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Tail(ctx context.Context, limit int32) ([]types.Row, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Tail{\n\t\t\tTail: &proto.Tail{\n\t\t\t\tInput: df.relation,\n\t\t\t\tLimit: limit,\n\t\t\t},\n\t\t},\n\t}\n\tdata := NewDataFrame(df.session, rel)\n\treturn data.Collect(ctx)\n}\n\nfunc (df *dataFrameImpl) Limit(ctx context.Context, limit int32) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Limit{\n\t\t\tLimit: &proto.Limit{\n\t\t\t\tInput: df.relation,\n\t\t\t\tLimit: limit,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Head(ctx context.Context, limit int32) ([]types.Row, error) {\n\treturn df.Limit(ctx, limit).Collect(ctx)\n}\n\nfunc (df *dataFrameImpl) Take(ctx context.Context, limit int32) ([]types.Row, error) {\n\treturn df.Limit(ctx, limit).Collect(ctx)\n}\n\nfunc (df *dataFrameImpl) ToArrow(ctx context.Context) (*arrow.Table, error) {\n\tresponseClient, err := df.session.client.ExecutePlan(ctx, df.createPlan())\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to execute plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\n\t_, table, err := responseClient.ToTable()\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\treturn &table, nil\n}\n\nfunc (df *dataFrameImpl) UnionAll(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tisAll := true\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:  df.relation,\n\t\t\t\tRightInput: otherDf.relation,\n\t\t\t\tSetOpType:  proto.SetOperation_SET_OP_TYPE_UNION,\n\t\t\t\tIsAll:      &isAll,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Union(ctx context.Context, other DataFrame) DataFrame {\n\treturn df.UnionAll(ctx, other)\n}\n\nfunc (df *dataFrameImpl) UnionByName(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tbyName := true\n\tallowMissingColumns := false\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:           df.relation,\n\t\t\t\tRightInput:          otherDf.relation,\n\t\t\t\tSetOpType:           proto.SetOperation_SET_OP_TYPE_UNION,\n\t\t\t\tByName:              &byName,\n\t\t\t\tAllowMissingColumns: &allowMissingColumns,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) UnionByNameWithMissingColumns(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tbyName := true\n\tallowMissingColumns := true\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:           df.relation,\n\t\t\t\tRightInput:          otherDf.relation,\n\t\t\t\tSetOpType:           proto.SetOperation_SET_OP_TYPE_UNION,\n\t\t\t\tByName:              &byName,\n\t\t\t\tAllowMissingColumns: &allowMissingColumns,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) ExceptAll(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tisAll := true\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:  df.relation,\n\t\t\t\tRightInput: otherDf.relation,\n\t\t\t\tSetOpType:  proto.SetOperation_SET_OP_TYPE_EXCEPT,\n\t\t\t\tIsAll:      &isAll,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Subtract(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tisAll := false\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:  df.relation,\n\t\t\t\tRightInput: otherDf.relation,\n\t\t\t\tSetOpType:  proto.SetOperation_SET_OP_TYPE_EXCEPT,\n\t\t\t\tIsAll:      &isAll,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Intersect(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tisAll := false\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:  df.relation,\n\t\t\t\tRightInput: otherDf.relation,\n\t\t\t\tSetOpType:  proto.SetOperation_SET_OP_TYPE_INTERSECT,\n\t\t\t\tIsAll:      &isAll,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) IntersectAll(ctx context.Context, other DataFrame) DataFrame {\n\totherDf := other.(*dataFrameImpl)\n\tisAll := true\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_SetOp{\n\t\t\tSetOp: &proto.SetOperation{\n\t\t\t\tLeftInput:  df.relation,\n\t\t\t\tRightInput: otherDf.relation,\n\t\t\t\tSetOpType:  proto.SetOperation_SET_OP_TYPE_INTERSECT,\n\t\t\t\tIsAll:      &isAll,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Sort(ctx context.Context, columns ...column.Convertible) (DataFrame, error) {\n\tglobalSort := true\n\tsortExprs := make([]*proto.Expression_SortOrder, 0, len(columns))\n\tfor _, c := range columns {\n\t\texpr, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tso := expr.GetSortOrder()\n\t\tif so == nil {\n\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\t\"sort expression must not be nil\"), sparkerrors.InvalidArgumentError)\n\t\t}\n\t\tsortExprs = append(sortExprs, so)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Sort{\n\t\t\tSort: &proto.Sort{\n\t\t\t\tInput:    df.relation,\n\t\t\t\tOrder:    sortExprs,\n\t\t\t\tIsGlobal: &globalSort,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) SortWithinPartitions(ctx context.Context, columns ...column.Convertible) (DataFrame, error) {\n\tglobalSort := false\n\tsortExprs := make([]*proto.Expression_SortOrder, 0, len(columns))\n\tfor _, c := range columns {\n\t\texpr, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tsortExprs = append(sortExprs, expr.GetSortOrder())\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Sort{\n\t\t\tSort: &proto.Sort{\n\t\t\t\tInput:    df.relation,\n\t\t\t\tOrder:    sortExprs,\n\t\t\t\tIsGlobal: &globalSort,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) OrderBy(ctx context.Context, columns ...column.Convertible) (DataFrame, error) {\n\treturn df.Sort(ctx, columns...)\n}\n\nfunc (df *dataFrameImpl) Explain(ctx context.Context, explainMode utils.ExplainMode) (string, error) {\n\tplan := df.createPlan()\n\n\tresponseClient, err := df.session.client.Explain(ctx, plan, explainMode)\n\tif err != nil {\n\t\treturn \"\", sparkerrors.WithType(fmt.Errorf(\"failed to execute plan: %w\", err), sparkerrors.ExecutionError)\n\t}\n\treturn responseClient.GetExplain().GetExplainString(), nil\n}\n\nfunc (df *dataFrameImpl) Persist(ctx context.Context, storageLevel utils.StorageLevel) error {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n\treturn df.session.client.Persist(ctx, plan, storageLevel)\n}\n\nfunc (df *dataFrameImpl) Cache(ctx context.Context) error {\n\treturn df.Persist(ctx, utils.StorageLevelMemoryOnly)\n}\n\nfunc (df *dataFrameImpl) Unpersist(ctx context.Context) error {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n\treturn df.session.client.Unpersist(ctx, plan)\n}\n\nfunc (df *dataFrameImpl) GetStorageLevel(ctx context.Context) (*utils.StorageLevel, error) {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n\treturn df.session.client.GetStorageLevel(ctx, plan)\n}\n\nfunc (df *dataFrameImpl) SameSemantics(ctx context.Context, other DataFrame) (bool, error) {\n\totherDf := other.(*dataFrameImpl)\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n\totherPlan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: otherDf.relation,\n\t\t},\n\t}\n\treturn df.session.client.SameSemantics(ctx, plan, otherPlan)\n}\n\nfunc (df *dataFrameImpl) SemanticHash(ctx context.Context) (int32, error) {\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Root{\n\t\t\tRoot: df.relation,\n\t\t},\n\t}\n\treturn df.session.client.SemanticHash(ctx, plan)\n}\n\nfunc (df *dataFrameImpl) Join(ctx context.Context, other DataFrame, onExpr column.Convertible, joinType utils.JoinType) (DataFrame, error) {\n\totherDf := other.(*dataFrameImpl)\n\tonExpression, err := onExpr.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\tjoinTypeProto := utils.ToProtoJoinType(joinType)\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Join{\n\t\t\tJoin: &proto.Join{\n\t\t\t\tLeft:          df.relation,\n\t\t\t\tRight:         otherDf.relation,\n\t\t\t\tJoinType:      joinTypeProto,\n\t\t\t\tJoinCondition: onExpression,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) CrossTab(ctx context.Context, col1, col2 string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_Crosstab{\n\t\t\tCrosstab: &proto.StatCrosstab{\n\t\t\t\tInput: df.relation,\n\t\t\t\tCol1:  col1,\n\t\t\t\tCol2:  col2,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Cube(ctx context.Context, cols ...column.Convertible) *GroupedData {\n\treturn &GroupedData{\n\t\tdf:           df,\n\t\tgroupingCols: cols,\n\t\tgroupType:    \"cube\",\n\t}\n}\n\nfunc (df *dataFrameImpl) Rollup(ctx context.Context, cols ...column.Convertible) *GroupedData {\n\treturn &GroupedData{\n\t\tdf:           df,\n\t\tgroupingCols: cols,\n\t\tgroupType:    \"rollup\",\n\t}\n}\n\nfunc (df *dataFrameImpl) Describe(ctx context.Context, cols ...string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_Describe{\n\t\t\tDescribe: &proto.StatDescribe{\n\t\t\t\tInput: df.relation,\n\t\t\t\tCols:  cols,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Distinct(ctx context.Context) DataFrame {\n\tallColumnsAsKeys := true\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Deduplicate{\n\t\t\tDeduplicate: &proto.Deduplicate{\n\t\t\t\tInput:            df.relation,\n\t\t\t\tAllColumnsAsKeys: &allColumnsAsKeys,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) First(ctx context.Context) (types.Row, error) {\n\trows, err := df.Head(ctx, 1)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn rows[0], nil\n}\n\nfunc (df *dataFrameImpl) FreqItems(ctx context.Context, cols ...string) DataFrame {\n\treturn df.FreqItemsWithSupport(ctx, 0.01, cols...)\n}\n\nfunc (df *dataFrameImpl) FreqItemsWithSupport(ctx context.Context, support float64, cols ...string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_FreqItems{\n\t\t\tFreqItems: &proto.StatFreqItems{\n\t\t\t\tInput:   df.relation,\n\t\t\t\tCols:    cols,\n\t\t\t\tSupport: &support,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) IsEmpty(ctx context.Context) (bool, error) {\n\td, err := df.Select(ctx)\n\tif err != nil {\n\t\treturn false, err\n\t}\n\trows, err := d.Take(ctx, int32(1))\n\tif err != nil {\n\t\treturn false, err\n\t}\n\treturn len(rows) == 0, nil\n}\n\nfunc (df *dataFrameImpl) Offset(ctx context.Context, offset int32) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_Offset{\n\t\t\tOffset: &proto.Offset{\n\t\t\t\tInput:  df.relation,\n\t\t\t\tOffset: offset,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) RandomSplit(ctx context.Context, weights []float64) ([]DataFrame, error) {\n\t// Check that we don't have negative weights:\n\ttotal := 0.0\n\tfor _, w := range weights {\n\t\tif w < 0.0 {\n\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"weights must not be negative\"), sparkerrors.InvalidArgumentError)\n\t\t}\n\t\ttotal += w\n\t}\n\tseed := rand.Int64()\n\tnormalizedWeights := make([]float64, len(weights))\n\tfor i, w := range weights {\n\t\tnormalizedWeights[i] = w / total\n\t}\n\n\t// Calculate the cumulative sum of the weights:\n\tcumulativeWeights := make([]float64, len(weights)+1)\n\tcumulativeWeights[0] = 0.0\n\tfor i := 0; i < len(normalizedWeights); i++ {\n\t\tcumulativeWeights[i+1] = cumulativeWeights[i] + normalizedWeights[i]\n\t}\n\n\t// Iterate over cumulative weights as the boundaries of the interval and create the dataframes:\n\tdataFrames := make([]DataFrame, len(weights))\n\twithReplacement := false\n\tfor i := 1; i < len(cumulativeWeights); i++ {\n\t\tsampleRelation := &proto.Relation{\n\t\t\tCommon: &proto.RelationCommon{\n\t\t\t\tPlanId: newPlanId(),\n\t\t\t},\n\t\t\tRelType: &proto.Relation_Sample{\n\t\t\t\tSample: &proto.Sample{\n\t\t\t\t\tInput:              df.relation,\n\t\t\t\t\tLowerBound:         cumulativeWeights[i-1],\n\t\t\t\t\tUpperBound:         cumulativeWeights[i],\n\t\t\t\t\tWithReplacement:    &withReplacement,\n\t\t\t\t\tSeed:               &seed,\n\t\t\t\t\tDeterministicOrder: true,\n\t\t\t\t},\n\t\t\t},\n\t\t}\n\t\tdataFrames[i-1] = NewDataFrame(df.session, sampleRelation)\n\t}\n\treturn dataFrames, nil\n}\n\nfunc (df *dataFrameImpl) Summary(ctx context.Context, statistics ...string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_Summary{\n\t\t\tSummary: &proto.StatSummary{\n\t\t\t\tInput:      df.relation,\n\t\t\t\tStatistics: statistics,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) Sample(ctx context.Context, fraction float64) (DataFrame, error) {\n\treturn df.sample(ctx, nil, fraction, nil)\n}\n\nfunc (df *dataFrameImpl) SampleWithReplacement(ctx context.Context, withReplacement bool, fraction float64) (DataFrame, error) {\n\treturn df.sample(ctx, &withReplacement, fraction, nil)\n}\n\nfunc (df *dataFrameImpl) SampleWithSeed(ctx context.Context, fraction float64, seed int64) (DataFrame, error) {\n\treturn df.sample(ctx, nil, fraction, &seed)\n}\n\nfunc (df *dataFrameImpl) SampleWithReplacementAndSeed(ctx context.Context, withReplacement bool, fraction float64, seed int64) (DataFrame, error) {\n\treturn df.sample(ctx, &withReplacement, fraction, &seed)\n}\n\nfunc (df *dataFrameImpl) sample(ctx context.Context, withReplacement *bool, fraction float64, seed *int64) (DataFrame, error) {\n\tif seed == nil {\n\t\tdefaultSeed := rand.Int64()\n\t\tseed = &defaultSeed\n\t}\n\n\tif withReplacement == nil {\n\t\tdefaultWithReplacement := false\n\t\twithReplacement = &defaultWithReplacement\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Sample{\n\t\t\tSample: &proto.Sample{\n\t\t\t\tInput:           df.relation,\n\t\t\t\tLowerBound:      0,\n\t\t\t\tUpperBound:      fraction,\n\t\t\t\tWithReplacement: withReplacement,\n\t\t\t\tSeed:            seed,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Replace(ctx context.Context,\n\ttoReplace []types.PrimitiveTypeLiteral, values []types.PrimitiveTypeLiteral, cols ...string,\n) (DataFrame, error) {\n\tif len(toReplace) != len(values) {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\"toReplace and values must have the same length\"), sparkerrors.InvalidArgumentError)\n\t}\n\n\ttoReplaceExprs := make([]*proto.Expression, 0, len(toReplace))\n\tfor _, c := range toReplace {\n\t\texpr, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\ttoReplaceExprs = append(toReplaceExprs, expr)\n\t}\n\n\tvaluesExprs := make([]*proto.Expression, 0, len(values))\n\tfor _, c := range values {\n\t\texpr, err := c.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tvaluesExprs = append(valuesExprs, expr)\n\t}\n\n\t// Create a list of NAReplace expressions.\n\treplacements := make([]*proto.NAReplace_Replacement, 0, len(toReplace))\n\tfor i := 0; i < len(toReplace); i++ {\n\t\treplacement := &proto.NAReplace_Replacement{\n\t\t\tOldValue: toReplaceExprs[i].GetLiteral(),\n\t\t\tNewValue: valuesExprs[i].GetLiteral(),\n\t\t}\n\t\treplacements = append(replacements, replacement)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Replace{\n\t\t\tReplace: &proto.NAReplace{\n\t\t\t\tInput:        df.relation,\n\t\t\t\tReplacements: replacements,\n\t\t\t\tCols:         cols,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Melt(ctx context.Context,\n\tids []column.Convertible,\n\tvalues []column.Convertible,\n\tvariableColumnName string,\n\tvalueColumnName string,\n) (DataFrame, error) {\n\treturn df.Unpivot(ctx, ids, values, variableColumnName, valueColumnName)\n}\n\nfunc (df *dataFrameImpl) Unpivot(ctx context.Context,\n\tids []column.Convertible,\n\tvalues []column.Convertible,\n\tvariableColumnName string,\n\tvalueColumnName string,\n) (DataFrame, error) {\n\tidExprs := make([]*proto.Expression, 0, len(ids))\n\tfor _, id := range ids {\n\t\texpr, err := id.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tidExprs = append(idExprs, expr)\n\t}\n\n\tvalueExprs := make([]*proto.Expression, 0, len(values))\n\tfor _, value := range values {\n\t\texpr, err := value.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tvalueExprs = append(valueExprs, expr)\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\n\t\tRelType: &proto.Relation_Unpivot{\n\t\t\tUnpivot: &proto.Unpivot{\n\t\t\t\tInput: df.relation,\n\t\t\t\tIds:   idExprs,\n\t\t\t\tValues: &proto.Unpivot_Values{\n\t\t\t\t\tValues: valueExprs,\n\t\t\t\t},\n\t\t\t\tVariableColumnName: variableColumnName,\n\t\t\t\tValueColumnName:    valueColumnName,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc makeDataframeWithFillNaRelation(df *dataFrameImpl, values []*proto.Expression_Literal, columns []string) DataFrame {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_FillNa{\n\t\t\tFillNa: &proto.NAFill{\n\t\t\t\tInput:  df.relation,\n\t\t\t\tCols:   columns,\n\t\t\t\tValues: values,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel)\n}\n\nfunc (df *dataFrameImpl) FillNa(ctx context.Context, value types.PrimitiveTypeLiteral, columns ...string) (DataFrame, error) {\n\tvalueLiteral, err := value.ToProto(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\treturn makeDataframeWithFillNaRelation(df, []*proto.Expression_Literal{\n\t\tvalueLiteral.GetLiteral(),\n\t}, columns), nil\n}\n\nfunc (df *dataFrameImpl) FillNaWithValues(ctx context.Context,\n\tvalues map[string]types.PrimitiveTypeLiteral,\n) (DataFrame, error) {\n\tvalueLiterals := make([]*proto.Expression_Literal, 0, len(values))\n\tcolumns := make([]string, 0, len(values))\n\tfor k, v := range values {\n\t\tvalueLiteral, err := v.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tvalueLiterals = append(valueLiterals, valueLiteral.GetLiteral())\n\t\tcolumns = append(columns, k)\n\t}\n\treturn makeDataframeWithFillNaRelation(df, valueLiterals, columns), nil\n}\n\nfunc (df *dataFrameImpl) Stat() DataFrameStatFunctions {\n\treturn &dataFrameStatFunctionsImpl{df: df}\n}\n\nfunc (df *dataFrameImpl) Agg(ctx context.Context, cols ...column.Convertible) (DataFrame, error) {\n\treturn df.GroupBy().Agg(ctx, cols...)\n}\n\nfunc (df *dataFrameImpl) AggWithMap(ctx context.Context, exprs map[string]string) (DataFrame, error) {\n\tfuns := make([]column.Convertible, 0)\n\tfor k, v := range exprs {\n\t\t// Convert the column name to a column expression.\n\t\tcol := column.OfDF(df, k)\n\t\t// Convert the value string to an unresolved function name.\n\t\tfun := column.NewUnresolvedFunctionWithColumns(v, col)\n\t\tfuns = append(funs, fun)\n\t}\n\treturn df.Agg(ctx, funs...)\n}\n\nfunc (df *dataFrameImpl) ApproxQuantile(ctx context.Context, probabilities []float64,\n\trelativeError float64, cols ...string,\n) ([][]float64, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_ApproxQuantile{\n\t\t\tApproxQuantile: &proto.StatApproxQuantile{\n\t\t\t\tInput:         df.relation,\n\t\t\t\tProbabilities: probabilities,\n\t\t\t\tRelativeError: relativeError,\n\t\t\t\tCols:          cols,\n\t\t\t},\n\t\t},\n\t}\n\tdata := NewDataFrame(df.session, rel)\n\trows, err := data.Collect(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\t// The result structure is a bit weird here, essentially it returns exactly one row with\n\t// the quantiles.\n\t// Inside the row is a list of nested arroys that contain the quantiles. The first column is the\n\t// first nested array, the second column is the second nested array and so on.\n\n\tnested := rows[0].At(0).([]interface{})\n\tresult := make([][]float64, len(nested))\n\tfor i := 0; i < len(nested); i++ {\n\t\ttmp := nested[i].([]interface{})\n\t\tresult[i] = make([]float64, len(tmp))\n\t\tfor j := 0; j < len(tmp); j++ {\n\t\t\tf, ok := tmp[j].(float64)\n\t\t\tif !ok {\n\t\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\t\t\"failed to cast to float64\"), sparkerrors.ExecutionError)\n\t\t\t}\n\t\t\tresult[i][j] = f\n\t\t}\n\t}\n\treturn result, nil\n}\n\nfunc (df *dataFrameImpl) DropNa(ctx context.Context, subset ...string) (DataFrame, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_DropNa{\n\t\t\tDropNa: &proto.NADrop{\n\t\t\t\tInput: df.relation,\n\t\t\t\tCols:  subset,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) DropNaAll(ctx context.Context, subset ...string) (DataFrame, error) {\n\treturn df.DropNaWithThreshold(ctx, 1, subset...)\n}\n\nfunc (df *dataFrameImpl) DropNaWithThreshold(ctx context.Context, thresh int32, subset ...string) (DataFrame, error) {\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_DropNa{\n\t\t\tDropNa: &proto.NADrop{\n\t\t\t\tInput:       df.relation,\n\t\t\t\tMinNonNulls: &thresh,\n\t\t\t\tCols:        subset,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(df.session, rel), nil\n}\n\nfunc (df *dataFrameImpl) Na() DataFrameNaFunctions {\n\treturn &dataFrameNaFunctionsImpl{dataFrame: df}\n}\n\nfunc (df *dataFrameImpl) All(ctx context.Context) iter.Seq2[types.Row, error] {\n\tdata, err := df.Collect(ctx)\n\treturn func(yield func(types.Row, error) bool) {\n\t\tif err != nil {\n\t\t\tyield(nil, err)\n\t\t\treturn\n\t\t}\n\t\tfor _, row := range data {\n\t\t\tif !yield(row, nil) {\n\t\t\t\tbreak\n\t\t\t}\n\t\t}\n\t}\n}\n\nfunc (df *dataFrameImpl) PrintSchema(ctx context.Context) error {\n\tschema, err := df.Schema(ctx)\n\tif err != nil {\n\t\treturn err\n\t}\n\tfmt.Print(schema.TreeString())\n\treturn nil\n}\n"
  },
  {
    "path": "spark/sql/dataframe_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestDataFrameImpl_GroupBy(t *testing.T) {\n\tctx := context.Background()\n\trel := &proto.Relation{\n\t\tRelType: &proto.Relation_Range{\n\t\t\tRange: &proto.Range{\n\t\t\t\tEnd:  10,\n\t\t\t\tStep: 1,\n\t\t\t},\n\t\t},\n\t}\n\tdf := NewDataFrame(nil, rel)\n\tgd := df.GroupBy(functions.Col(\"id\"))\n\tassert.NotNil(t, gd)\n\n\tassert.Equal(t, gd.groupType, \"groupby\")\n\n\tdf, err := gd.Agg(ctx, functions.Count(functions.Int64Lit(1)))\n\tassert.Nil(t, err)\n\timpl := df.(*dataFrameImpl)\n\tassert.NotNil(t, impl)\n\tassert.IsType(t, impl.relation.RelType, &proto.Relation_Aggregate{})\n}\n"
  },
  {
    "path": "spark/sql/dataframenafunctions.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n)\n\ntype DataFrameNaFunctions interface {\n\tDrop(ctx context.Context, cols ...string) (DataFrame, error)\n\tDropAll(ctx context.Context, cols ...string) (DataFrame, error)\n\tDropWithThreshold(ctx context.Context, threshold int32, cols ...string) (DataFrame, error)\n\tFill(ctx context.Context, value types.PrimitiveTypeLiteral, cols ...string) (DataFrame, error)\n\tFillWithValues(ctx context.Context, values map[string]types.PrimitiveTypeLiteral) (DataFrame, error)\n\tReplace(ctx context.Context, toReplace []types.PrimitiveTypeLiteral,\n\t\tvalues []types.PrimitiveTypeLiteral, cols ...string) (DataFrame, error)\n}\n\ntype dataFrameNaFunctionsImpl struct {\n\tdataFrame DataFrame\n}\n\nfunc (d *dataFrameNaFunctionsImpl) Drop(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn d.dataFrame.DropNa(ctx, cols...)\n}\n\nfunc (d *dataFrameNaFunctionsImpl) DropAll(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn d.dataFrame.DropNaAll(ctx, cols...)\n}\n\nfunc (d *dataFrameNaFunctionsImpl) DropWithThreshold(ctx context.Context, threshold int32, cols ...string) (DataFrame, error) {\n\treturn d.dataFrame.DropNaWithThreshold(ctx, threshold, cols...)\n}\n\nfunc (d *dataFrameNaFunctionsImpl) Fill(ctx context.Context, value types.PrimitiveTypeLiteral, cols ...string) (DataFrame, error) {\n\treturn d.dataFrame.FillNa(ctx, value, cols...)\n}\n\nfunc (d *dataFrameNaFunctionsImpl) FillWithValues(ctx context.Context,\n\tvalues map[string]types.PrimitiveTypeLiteral,\n) (DataFrame, error) {\n\treturn d.dataFrame.FillNaWithValues(ctx, values)\n}\n\nfunc (d *dataFrameNaFunctionsImpl) Replace(ctx context.Context,\n\ttoReplace []types.PrimitiveTypeLiteral, values []types.PrimitiveTypeLiteral, cols ...string,\n) (DataFrame, error) {\n\treturn d.dataFrame.Replace(ctx, toReplace, values, cols...)\n}\n"
  },
  {
    "path": "spark/sql/dataframereader.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\n// DataFrameReader supports reading data from storage and returning a data frame.\n// TODO needs to implement other methods like Option(), Schema(), and also \"strong typed\"\n// reading (e.g. Parquet(), Orc(), Csv(), etc.\ntype DataFrameReader interface {\n\t// Format specifies data format (data source type) for the underlying data, e.g. parquet.\n\tFormat(source string) DataFrameReader\n\t// Load reads the underlying data and returns a data frame.\n\tLoad(path string) (DataFrame, error)\n\t// Reads a table from the underlying data source.\n\tTable(name string) (DataFrame, error)\n\tOption(key, value string) DataFrameReader\n}\n\n// dataFrameReaderImpl is an implementation of DataFrameReader interface.\ntype dataFrameReaderImpl struct {\n\tsparkSession *sparkSessionImpl\n\tformatSource string\n\toptions      map[string]string\n}\n\n// NewDataframeReader creates a new DataFrameReader\nfunc NewDataframeReader(session *sparkSessionImpl) DataFrameReader {\n\treturn &dataFrameReaderImpl{\n\t\tsparkSession: session,\n\t}\n}\n\nfunc (w *dataFrameReaderImpl) Table(name string) (DataFrame, error) {\n\treturn NewDataFrame(w.sparkSession, newReadTableRelation(name)), nil\n}\n\nfunc (w *dataFrameReaderImpl) Format(source string) DataFrameReader {\n\tw.formatSource = source\n\treturn w\n}\n\nfunc (w *dataFrameReaderImpl) Load(path string) (DataFrame, error) {\n\tvar format string\n\tif w.formatSource != \"\" {\n\t\tformat = w.formatSource\n\t}\n\tif w.options == nil {\n\t\treturn NewDataFrame(w.sparkSession, newReadWithFormatAndPath(path, format)), nil\n\t}\n\treturn NewDataFrame(w.sparkSession, newReadWithFormatAndPathAndOptions(path, format, w.options)), nil\n}\n\nfunc (w *dataFrameReaderImpl) Option(key, value string) DataFrameReader {\n\tif w.options == nil {\n\t\tw.options = make(map[string]string)\n\t}\n\tw.options[key] = value\n\treturn w\n}\n"
  },
  {
    "path": "spark/sql/dataframereader_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"testing\"\n\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestLoadCreatesADataFrame(t *testing.T) {\n\treader := NewDataframeReader(nil)\n\tsource := \"source\"\n\tpath := \"path\"\n\treader.Format(source)\n\tframe, err := reader.Load(path)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, frame)\n}\n\nfunc TestRelationContainsPathAndFormat(t *testing.T) {\n\tformatSource := \"source\"\n\tpath := \"path\"\n\trelation := newReadWithFormatAndPath(path, formatSource)\n\tassert.NotNil(t, relation)\n\tassert.Equal(t, &formatSource, relation.GetRead().GetDataSource().Format)\n\tassert.Equal(t, path, relation.GetRead().GetDataSource().Paths[0])\n}\n\nfunc TestRelationContainsPathAndFormatAndOptions(t *testing.T) {\n\tformatSource := \"source\"\n\tpath := \"path\"\n\toptions := map[string]string{\"key\": \"value\"}\n\trelation := newReadWithFormatAndPathAndOptions(path, formatSource, options)\n\tassert.NotNil(t, relation)\n\tassert.Equal(t, &formatSource, relation.GetRead().GetDataSource().Format)\n\tassert.Equal(t, path, relation.GetRead().GetDataSource().Paths[0])\n\tfor i, v := range options {\n\t\tassert.Equal(t, v, relation.GetRead().GetDataSource().Options[i])\n\t}\n}\n"
  },
  {
    "path": "spark/sql/dataframestatfunctions.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport \"context\"\n\ntype DataFrameStatFunctions interface {\n\tApproxQuantile(ctx context.Context, probabilities []float64, relativeError float64, cols ...string) ([][]float64, error)\n\tCov(ctx context.Context, col1, col2 string) (float64, error)\n\tCorr(ctx context.Context, col1, col2 string) (float64, error)\n\tCorrWithMethod(ctx context.Context, col1, col2 string, method string) (float64, error)\n\tCrossTab(ctx context.Context, col1, col2 string) DataFrame\n\tFreqItems(ctx context.Context, cols ...string) DataFrame\n\tFreqItemsWithSupport(ctx context.Context, support float64, cols ...string) DataFrame\n\tSample(ctx context.Context, fraction float64) (DataFrame, error)\n\tSampleWithReplacement(ctx context.Context, withReplacement bool, fraction float64) (DataFrame, error)\n\tSampleWithSeed(ctx context.Context, fraction float64, seed int64) (DataFrame, error)\n\tSampleWithReplacementAndSeed(ctx context.Context, withReplacement bool, fraction float64, seed int64) (DataFrame, error)\n}\n\ntype dataFrameStatFunctionsImpl struct {\n\tdf DataFrame\n}\n\nfunc (d *dataFrameStatFunctionsImpl) Sample(ctx context.Context, fraction float64) (DataFrame, error) {\n\treturn d.df.Sample(ctx, fraction)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) SampleWithReplacement(ctx context.Context,\n\twithReplacement bool, fraction float64,\n) (DataFrame, error) {\n\treturn d.df.SampleWithReplacement(ctx, withReplacement, fraction)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) SampleWithSeed(ctx context.Context, fraction float64, seed int64) (DataFrame, error) {\n\treturn d.df.SampleWithSeed(ctx, fraction, seed)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) SampleWithReplacementAndSeed(ctx context.Context,\n\twithReplacement bool, fraction float64, seed int64,\n) (DataFrame, error) {\n\treturn d.df.SampleWithReplacementAndSeed(ctx, withReplacement, fraction, seed)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) ApproxQuantile(ctx context.Context, probabilities []float64,\n\trelativeError float64, cols ...string,\n) ([][]float64, error) {\n\treturn d.df.ApproxQuantile(ctx, probabilities, relativeError, cols...)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) Cov(ctx context.Context, col1, col2 string) (float64, error) {\n\treturn d.df.Cov(ctx, col1, col2)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) Corr(ctx context.Context, col1, col2 string) (float64, error) {\n\treturn d.df.Corr(ctx, col1, col2)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) CorrWithMethod(ctx context.Context, col1, col2 string, method string) (float64, error) {\n\treturn d.df.CorrWithMethod(ctx, col1, col2, method)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) CrossTab(ctx context.Context, col1, col2 string) DataFrame {\n\treturn d.df.CrossTab(ctx, col1, col2)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) FreqItems(ctx context.Context, cols ...string) DataFrame {\n\treturn d.df.FreqItems(ctx, cols...)\n}\n\nfunc (d *dataFrameStatFunctionsImpl) FreqItemsWithSupport(ctx context.Context, support float64, cols ...string) DataFrame {\n\treturn d.df.FreqItemsWithSupport(ctx, support, cols...)\n}\n"
  },
  {
    "path": "spark/sql/dataframewriter.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\t\"fmt\"\n\t\"strings\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\n// DataFrameWriter supports writing data frame to storage.\ntype DataFrameWriter interface {\n\t// Mode specifies saving mode for the data, e.g. Append, Overwrite, ErrorIfExists.\n\tMode(saveMode string) DataFrameWriter\n\t// Format specifies data format (data source type) for the underlying data, e.g. parquet.\n\tFormat(source string) DataFrameWriter\n\t// Save writes data frame to the given path.\n\tSave(ctx context.Context, path string) error\n}\n\nfunc newDataFrameWriter(sparkExecutor *sparkSessionImpl, relation *proto.Relation) DataFrameWriter {\n\treturn &dataFrameWriterImpl{\n\t\tsparkExecutor: sparkExecutor,\n\t\trelation:      relation,\n\t}\n}\n\n// dataFrameWriterImpl is an implementation of DataFrameWriter interface.\ntype dataFrameWriterImpl struct {\n\tsparkExecutor *sparkSessionImpl\n\trelation      *proto.Relation\n\tsaveMode      string\n\tformatSource  string\n}\n\nfunc (w *dataFrameWriterImpl) Mode(saveMode string) DataFrameWriter {\n\tw.saveMode = saveMode\n\treturn w\n}\n\nfunc (w *dataFrameWriterImpl) Format(source string) DataFrameWriter {\n\tw.formatSource = source\n\treturn w\n}\n\nfunc (w *dataFrameWriterImpl) Save(ctx context.Context, path string) error {\n\tsaveMode, err := getSaveMode(w.saveMode)\n\tif err != nil {\n\t\treturn err\n\t}\n\tvar source *string\n\tif w.formatSource != \"\" {\n\t\tsource = &w.formatSource\n\t}\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Command{\n\t\t\tCommand: &proto.Command{\n\t\t\t\tCommandType: &proto.Command_WriteOperation{\n\t\t\t\t\tWriteOperation: &proto.WriteOperation{\n\t\t\t\t\t\tInput:  w.relation,\n\t\t\t\t\t\tMode:   saveMode,\n\t\t\t\t\t\tSource: source,\n\t\t\t\t\t\tSaveType: &proto.WriteOperation_Path{\n\t\t\t\t\t\t\tPath: path,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tresponseClient, err := w.sparkExecutor.client.ExecutePlan(ctx, plan)\n\tif err != nil {\n\t\treturn err\n\t}\n\n\t_, _, err = responseClient.ToTable()\n\treturn err\n}\n\nfunc getSaveMode(mode string) (proto.WriteOperation_SaveMode, error) {\n\tif mode == \"\" {\n\t\treturn proto.WriteOperation_SAVE_MODE_UNSPECIFIED, nil\n\t} else if strings.EqualFold(mode, \"Append\") {\n\t\treturn proto.WriteOperation_SAVE_MODE_APPEND, nil\n\t} else if strings.EqualFold(mode, \"Overwrite\") {\n\t\treturn proto.WriteOperation_SAVE_MODE_OVERWRITE, nil\n\t} else if strings.EqualFold(mode, \"ErrorIfExists\") {\n\t\treturn proto.WriteOperation_SAVE_MODE_ERROR_IF_EXISTS, nil\n\t} else if strings.EqualFold(mode, \"Ignore\") {\n\t\treturn proto.WriteOperation_SAVE_MODE_IGNORE, nil\n\t} else {\n\t\treturn 0, sparkerrors.WithType(fmt.Errorf(\"unsupported save mode: %s\", mode), sparkerrors.InvalidInputError)\n\t}\n}\n"
  },
  {
    "path": "spark/sql/dataframewriter_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\t\"github.com/apache/spark-connect-go/spark/client\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestGetSaveMode(t *testing.T) {\n\tmode, err := getSaveMode(\"\")\n\tassert.Nil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_UNSPECIFIED, mode)\n\n\tmode, err = getSaveMode(\"append\")\n\tassert.Nil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_APPEND, mode)\n\n\tmode, err = getSaveMode(\"Overwrite\")\n\tassert.Nil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_OVERWRITE, mode)\n\n\tmode, err = getSaveMode(\"ErrorIfExists\")\n\tassert.Nil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_ERROR_IF_EXISTS, mode)\n\n\tmode, err = getSaveMode(\"IGNORE\")\n\tassert.Nil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_IGNORE, mode)\n\n\tmode, err = getSaveMode(\"XYZ\")\n\tassert.NotNil(t, err)\n\tassert.Equal(t, proto.WriteOperation_SAVE_MODE_UNSPECIFIED, mode)\n}\n\nfunc TestSaveExecutesWriteOperationUntilEOF(t *testing.T) {\n\trelation := &proto.Relation{}\n\texecutor := client.NewTestConnectClientFromResponses(mocks.MockSessionId,\n\t\t&mocks.ExecutePlanResponseDone, &mocks.ExecutePlanResponseEOF)\n\tsession := &sparkSessionImpl{\n\t\tclient:    executor,\n\t\tsessionId: mocks.MockSessionId,\n\t}\n\tctx := context.Background()\n\tpath := \"path\"\n\n\twriter := newDataFrameWriter(session, relation)\n\twriter.Format(\"format\")\n\twriter.Mode(\"append\")\n\terr := writer.Save(ctx, path)\n\tassert.NoError(t, err)\n}\n\nfunc TestSaveFailsIfAnotherErrorHappensWhenReadingStream(t *testing.T) {\n\trelation := &proto.Relation{}\n\texecutor := client.NewTestConnectClientFromResponses(mocks.MockSessionId, &mocks.MockResponse{\n\t\tErr: assert.AnError,\n\t})\n\tsession := &sparkSessionImpl{\n\t\tclient:    executor,\n\t\tsessionId: mocks.MockSessionId,\n\t}\n\tctx := context.Background()\n\tpath := \"path\"\n\n\twriter := newDataFrameWriter(session, relation)\n\twriter.Format(\"format\")\n\twriter.Mode(\"append\")\n\terr := writer.Save(ctx, path)\n\tassert.Error(t, err)\n}\n\nfunc TestSaveFailsIfAnotherErrorHappensWhenExecuting(t *testing.T) {\n\trelation := &proto.Relation{}\n\texecutor := client.NewTestConnectClientWithImmediateError(mocks.MockSessionId, assert.AnError)\n\tsession := &sparkSessionImpl{\n\t\tclient: executor,\n\t}\n\tctx := context.Background()\n\tpath := \"path\"\n\n\twriter := newDataFrameWriter(session, relation)\n\twriter.Format(\"format\")\n\twriter.Mode(\"append\")\n\terr := writer.Save(ctx, path)\n\tassert.Error(t, err)\n}\n"
  },
  {
    "path": "spark/sql/executeplanclient.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n"
  },
  {
    "path": "spark/sql/functions/buiitins.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage functions\n\nimport (\n\t\"github.com/apache/spark-connect-go/spark/sql/column\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n)\n\nfunc Expr(expr string) column.Column {\n\treturn column.NewColumn(column.NewSQLExpression(expr))\n}\n\nfunc Col(name string) column.Column {\n\treturn column.NewColumn(column.NewColumnReference(name))\n}\n\nfunc Lit(value types.LiteralType) column.Column {\n\treturn column.NewColumn(column.NewLiteral(value))\n}\n\nfunc Int8Lit(value int8) column.Column {\n\treturn Lit(types.Int8(value))\n}\n\nfunc Int16Lit(value int16) column.Column {\n\treturn Lit(types.Int16(value))\n}\n\nfunc Int32Lit(value int32) column.Column {\n\treturn Lit(types.Int32(value))\n}\n\nfunc Int64Lit(value int64) column.Column {\n\treturn Lit(types.Int64(value))\n}\n\nfunc Float32Lit(value float32) column.Column {\n\treturn Lit(types.Float32(value))\n}\n\nfunc Float64Lit(value float64) column.Column {\n\treturn Lit(types.Float64(value))\n}\n\nfunc StringLit(value string) column.Column {\n\treturn Lit(types.String(value))\n}\n\nfunc BoolLit(value bool) column.Column {\n\treturn Lit(types.Boolean(value))\n}\n\nfunc BinaryLit(value []byte) column.Column {\n\treturn Lit(types.Binary(value))\n}\n\nfunc IntLit(value int) column.Column {\n\treturn Lit(types.Int(value))\n}\n"
  },
  {
    "path": "spark/sql/functions/generated.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage functions\n\nimport \"github.com/apache/spark-connect-go/spark/sql/column\"\n\n// BitwiseNOT - Computes bitwise not.\n//\n// BitwiseNOT is the Golang equivalent of bitwiseNOT: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitwiseNOT(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitwiseNOT\", col))\n}\n\n// BitwiseNot - Computes bitwise not.\n//\n// BitwiseNot is the Golang equivalent of bitwise_not: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitwiseNot(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitwise_not\", col))\n}\n\n// BitCount - Returns the number of bits that are set in the argument expr as an unsigned 64-bit integer,\n// or NULL if the argument is NULL.\n//\n// BitCount is the Golang equivalent of bit_count: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitCount(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_count\", col))\n}\n\n// BitGet - Returns the value of the bit (0 or 1) at the specified position.\n// The positions are numbered from right to left, starting at zero.\n// The position argument cannot be negative.\n//\n// BitGet is the Golang equivalent of bit_get: (col: 'ColumnOrName', pos: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitGet(col column.Column, pos column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_get\", col, pos))\n}\n\n// Getbit - Returns the value of the bit (0 or 1) at the specified position.\n// The positions are numbered from right to left, starting at zero.\n// The position argument cannot be negative.\n//\n// Getbit is the Golang equivalent of getbit: (col: 'ColumnOrName', pos: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Getbit(col column.Column, pos column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"getbit\", col, pos))\n}\n\n// TODO: broadcast: (df: 'DataFrame') -> 'DataFrame'\n\n// Coalesce - Returns the first column that is not null.\n//\n// Coalesce is the Golang equivalent of coalesce: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Coalesce(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"coalesce\", vals...))\n}\n\n// Greatest - Returns the greatest value of the list of column names, skipping null values.\n// This function takes at least 2 parameters. It will return null if all parameters are null.\n//\n// Greatest is the Golang equivalent of greatest: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Greatest(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"greatest\", vals...))\n}\n\n// InputFileName - Creates a string column for the file name of the current Spark task.\n//\n// InputFileName is the Golang equivalent of input_file_name: () -> pyspark.sql.connect.column.Column\nfunc InputFileName() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"input_file_name\"))\n}\n\n// Least - Returns the least value of the list of column names, skipping null values.\n// This function takes at least 2 parameters. It will return null if all parameters are null.\n//\n// Least is the Golang equivalent of least: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Least(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"least\", vals...))\n}\n\n// Isnan - An expression that returns true if the column is NaN.\n//\n// Isnan is the Golang equivalent of isnan: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Isnan(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"isnan\", col))\n}\n\n// Isnull - An expression that returns true if the column is null.\n//\n// Isnull is the Golang equivalent of isnull: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Isnull(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"isnull\", col))\n}\n\n// MonotonicallyIncreasingId - A column that generates monotonically increasing 64-bit integers.\n//\n// The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive.\n// The current implementation puts the partition ID in the upper 31 bits, and the record number\n// within each partition in the lower 33 bits. The assumption is that the data frame has\n// less than 1 billion partitions, and each partition has less than 8 billion records.\n//\n// MonotonicallyIncreasingId is the Golang equivalent of monotonically_increasing_id: () -> pyspark.sql.connect.column.Column\nfunc MonotonicallyIncreasingId() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"monotonically_increasing_id\"))\n}\n\n// Nanvl - Returns col1 if it is not NaN, or col2 if col1 is NaN.\n//\n// Both inputs should be floating point columns (:class:`DoubleType` or :class:`FloatType`).\n//\n// Nanvl is the Golang equivalent of nanvl: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Nanvl(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"nanvl\", col1, col2))\n}\n\n// Rand - Generates a random column with independent and identically distributed (i.i.d.) samples\n// uniformly distributed in [0.0, 1.0).\n//\n// Rand is the Golang equivalent of rand: (seed: Optional[int] = None) -> pyspark.sql.connect.column.Column\nfunc Rand(seed int64) column.Column {\n\tlit_seed := Int64Lit(seed)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rand\", lit_seed))\n}\n\n// Randn - Generates a column with independent and identically distributed (i.i.d.) samples from\n// the standard normal distribution.\n//\n// Randn is the Golang equivalent of randn: (seed: Optional[int] = None) -> pyspark.sql.connect.column.Column\nfunc Randn(seed int64) column.Column {\n\tlit_seed := Int64Lit(seed)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"randn\", lit_seed))\n}\n\n// SparkPartitionId - A column for partition ID.\n//\n// SparkPartitionId is the Golang equivalent of spark_partition_id: () -> pyspark.sql.connect.column.Column\nfunc SparkPartitionId() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"spark_partition_id\"))\n}\n\n// TODO: when: (condition: pyspark.sql.connect.column.Column, value: Any) -> pyspark.sql.connect.column.Column\n\n// Asc - Returns a sort expression based on the ascending order of the given column name.\n//\n// Asc is the Golang equivalent of asc: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Asc(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"asc\", col))\n}\n\n// AscNullsFirst - Returns a sort expression based on the ascending order of the given\n// column name, and null values return before non-null values.\n//\n// AscNullsFirst is the Golang equivalent of asc_nulls_first: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc AscNullsFirst(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"asc_nulls_first\", col))\n}\n\n// AscNullsLast - Returns a sort expression based on the ascending order of the given\n// column name, and null values appear after non-null values.\n//\n// AscNullsLast is the Golang equivalent of asc_nulls_last: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc AscNullsLast(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"asc_nulls_last\", col))\n}\n\n// Desc - Returns a sort expression based on the descending order of the given column name.\n//\n// Desc is the Golang equivalent of desc: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Desc(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"desc\", col))\n}\n\n// DescNullsFirst - Returns a sort expression based on the descending order of the given\n// column name, and null values appear before non-null values.\n//\n// DescNullsFirst is the Golang equivalent of desc_nulls_first: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DescNullsFirst(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"desc_nulls_first\", col))\n}\n\n// DescNullsLast - Returns a sort expression based on the descending order of the given\n// column name, and null values appear after non-null values.\n//\n// DescNullsLast is the Golang equivalent of desc_nulls_last: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DescNullsLast(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"desc_nulls_last\", col))\n}\n\n// Abs - Computes the absolute value.\n//\n// Abs is the Golang equivalent of abs: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Abs(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"abs\", col))\n}\n\n// Acos - Computes inverse cosine of the input column.\n//\n// Acos is the Golang equivalent of acos: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Acos(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"acos\", col))\n}\n\n// Acosh - Computes inverse hyperbolic cosine of the input column.\n//\n// Acosh is the Golang equivalent of acosh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Acosh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"acosh\", col))\n}\n\n// Asin - Computes inverse sine of the input column.\n//\n// Asin is the Golang equivalent of asin: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Asin(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"asin\", col))\n}\n\n// Asinh - Computes inverse hyperbolic sine of the input column.\n//\n// Asinh is the Golang equivalent of asinh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Asinh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"asinh\", col))\n}\n\n// Atan - Compute inverse tangent of the input column.\n//\n// Atan is the Golang equivalent of atan: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Atan(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"atan\", col))\n}\n\n// Atan2 -\n//\n// Atan2 is the Golang equivalent of atan2: (col1: Union[ForwardRef('ColumnOrName'), float], col2: Union[ForwardRef('ColumnOrName'), float]) -> pyspark.sql.connect.column.Column\nfunc Atan2(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"atan2\", col1, col2))\n}\n\n// Atanh - Computes inverse hyperbolic tangent of the input column.\n//\n// Atanh is the Golang equivalent of atanh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Atanh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"atanh\", col))\n}\n\n// Bin - Returns the string representation of the binary value of the given column.\n//\n// Bin is the Golang equivalent of bin: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Bin(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bin\", col))\n}\n\n// Bround - Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0\n// or at integral part when `scale` < 0.\n//\n// Bround is the Golang equivalent of bround: (col: 'ColumnOrName', scale: int = 0) -> pyspark.sql.connect.column.Column\nfunc Bround(col column.Column, scale int64) column.Column {\n\tlit_scale := Int64Lit(scale)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bround\", col, lit_scale))\n}\n\n// Cbrt - Computes the cube-root of the given value.\n//\n// Cbrt is the Golang equivalent of cbrt: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Cbrt(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cbrt\", col))\n}\n\n// Ceil - Computes the ceiling of the given value.\n//\n// Ceil is the Golang equivalent of ceil: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ceil(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ceil\", col))\n}\n\n// Ceiling - Computes the ceiling of the given value.\n//\n// Ceiling is the Golang equivalent of ceiling: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ceiling(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ceiling\", col))\n}\n\n// Conv - Convert a number in a string column from one base to another.\n//\n// Conv is the Golang equivalent of conv: (col: 'ColumnOrName', fromBase: int, toBase: int) -> pyspark.sql.connect.column.Column\nfunc Conv(col column.Column, fromBase int64, toBase int64) column.Column {\n\tlit_fromBase := Int64Lit(fromBase)\n\tlit_toBase := Int64Lit(toBase)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"conv\", col, lit_fromBase, lit_toBase))\n}\n\n// Cos - Computes cosine of the input column.\n//\n// Cos is the Golang equivalent of cos: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Cos(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cos\", col))\n}\n\n// Cosh - Computes hyperbolic cosine of the input column.\n//\n// Cosh is the Golang equivalent of cosh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Cosh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cosh\", col))\n}\n\n// Cot - Computes cotangent of the input column.\n//\n// Cot is the Golang equivalent of cot: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Cot(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cot\", col))\n}\n\n// Csc - Computes cosecant of the input column.\n//\n// Csc is the Golang equivalent of csc: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Csc(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"csc\", col))\n}\n\n// Degrees - Converts an angle measured in radians to an approximately equivalent angle\n// measured in degrees.\n//\n// Degrees is the Golang equivalent of degrees: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Degrees(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"degrees\", col))\n}\n\n// E - Returns Euler's number.\n//\n// E is the Golang equivalent of e: () -> pyspark.sql.connect.column.Column\nfunc E() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"e\"))\n}\n\n// Exp - Computes the exponential of the given value.\n//\n// Exp is the Golang equivalent of exp: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Exp(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"exp\", col))\n}\n\n// Expm1 - Computes the exponential of the given value minus one.\n//\n// Expm1 is the Golang equivalent of expm1: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Expm1(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"expm1\", col))\n}\n\n// Factorial - Computes the factorial of the given value.\n//\n// Factorial is the Golang equivalent of factorial: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Factorial(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"factorial\", col))\n}\n\n// Floor - Computes the floor of the given value.\n//\n// Floor is the Golang equivalent of floor: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Floor(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"floor\", col))\n}\n\n// Hex - Computes hex value of the given column, which could be :class:`pyspark.sql.types.StringType`,\n// :class:`pyspark.sql.types.BinaryType`, :class:`pyspark.sql.types.IntegerType` or\n// :class:`pyspark.sql.types.LongType`.\n//\n// Hex is the Golang equivalent of hex: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Hex(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hex\", col))\n}\n\n// Hypot - Computes “sqrt(a^2 + b^2)“ without intermediate overflow or underflow.\n//\n// Hypot is the Golang equivalent of hypot: (col1: Union[ForwardRef('ColumnOrName'), float], col2: Union[ForwardRef('ColumnOrName'), float]) -> pyspark.sql.connect.column.Column\nfunc Hypot(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hypot\", col1, col2))\n}\n\n// Log - Returns the first argument-based logarithm of the second argument.\n//\n// If there is only one argument, then this takes the natural logarithm of the argument.\n//\n// Log is the Golang equivalent of log: (arg1: Union[ForwardRef('ColumnOrName'), float], arg2: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Log(arg1 column.Column, arg2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"log\", arg1, arg2))\n}\n\n// Log10 - Computes the logarithm of the given value in Base 10.\n//\n// Log10 is the Golang equivalent of log10: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Log10(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"log10\", col))\n}\n\n// Log1p - Computes the natural logarithm of the \"given value plus one\".\n//\n// Log1p is the Golang equivalent of log1p: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Log1p(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"log1p\", col))\n}\n\n// Ln - Returns the natural logarithm of the argument.\n//\n// Ln is the Golang equivalent of ln: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ln(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ln\", col))\n}\n\n// Log2 - Returns the base-2 logarithm of the argument.\n//\n// Log2 is the Golang equivalent of log2: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Log2(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"log2\", col))\n}\n\n// Negative - Returns the negative value.\n//\n// Negative is the Golang equivalent of negative: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Negative(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"negative\", col))\n}\n\n// Negate - Returns the negative value.\n//\n// Negate is the Golang equivalent of negate: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Negate(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"negate\", col))\n}\n\n// Pi - Returns Pi.\n//\n// Pi is the Golang equivalent of pi: () -> pyspark.sql.connect.column.Column\nfunc Pi() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"pi\"))\n}\n\n// Positive - Returns the value.\n//\n// Positive is the Golang equivalent of positive: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Positive(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"positive\", col))\n}\n\n// Pmod - Returns the positive value of dividend mod divisor.\n//\n// Pmod is the Golang equivalent of pmod: (dividend: Union[ForwardRef('ColumnOrName'), float], divisor: Union[ForwardRef('ColumnOrName'), float]) -> pyspark.sql.connect.column.Column\nfunc Pmod(dividend column.Column, divisor column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"pmod\", dividend, divisor))\n}\n\n// WidthBucket - Returns the bucket number into which the value of this expression would fall\n// after being evaluated. Note that input arguments must follow conditions listed below;\n// otherwise, the method will return null.\n//\n// WidthBucket is the Golang equivalent of width_bucket: (v: 'ColumnOrName', min: 'ColumnOrName', max: 'ColumnOrName', numBucket: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc WidthBucket(v column.Column, min column.Column, max column.Column, numBucket column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"width_bucket\", v, min, max, numBucket))\n}\n\n// Pow - Returns the value of the first argument raised to the power of the second argument.\n//\n// Pow is the Golang equivalent of pow: (col1: Union[ForwardRef('ColumnOrName'), float], col2: Union[ForwardRef('ColumnOrName'), float]) -> pyspark.sql.connect.column.Column\nfunc Pow(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"pow\", col1, col2))\n}\n\n// Radians - Converts an angle measured in degrees to an approximately equivalent angle\n// measured in radians.\n//\n// Radians is the Golang equivalent of radians: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Radians(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"radians\", col))\n}\n\n// Rint - Returns the double value that is closest in value to the argument and\n// is equal to a mathematical integer.\n//\n// Rint is the Golang equivalent of rint: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Rint(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rint\", col))\n}\n\n// Round - Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0\n// or at integral part when `scale` < 0.\n//\n// Round is the Golang equivalent of round: (col: 'ColumnOrName', scale: int = 0) -> pyspark.sql.connect.column.Column\nfunc Round(col column.Column, scale int64) column.Column {\n\tlit_scale := Int64Lit(scale)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"round\", col, lit_scale))\n}\n\n// Sec - Computes secant of the input column.\n//\n// Sec is the Golang equivalent of sec: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sec(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sec\", col))\n}\n\n// ShiftLeft - Shift the given value numBits left.\n//\n// ShiftLeft is the Golang equivalent of shiftLeft: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc ShiftLeft(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftLeft\", col, lit_numBits))\n}\n\n// Shiftleft - Shift the given value numBits left.\n//\n// Shiftleft is the Golang equivalent of shiftleft: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc Shiftleft(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftleft\", col, lit_numBits))\n}\n\n// ShiftRight - (Signed) shift the given value numBits right.\n//\n// ShiftRight is the Golang equivalent of shiftRight: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc ShiftRight(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftRight\", col, lit_numBits))\n}\n\n// Shiftright - (Signed) shift the given value numBits right.\n//\n// Shiftright is the Golang equivalent of shiftright: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc Shiftright(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftright\", col, lit_numBits))\n}\n\n// ShiftRightUnsigned - Unsigned shift the given value numBits right.\n//\n// ShiftRightUnsigned is the Golang equivalent of shiftRightUnsigned: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc ShiftRightUnsigned(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftRightUnsigned\", col, lit_numBits))\n}\n\n// Shiftrightunsigned - Unsigned shift the given value numBits right.\n//\n// Shiftrightunsigned is the Golang equivalent of shiftrightunsigned: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc Shiftrightunsigned(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shiftrightunsigned\", col, lit_numBits))\n}\n\n// Signum - Computes the signum of the given value.\n//\n// Signum is the Golang equivalent of signum: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Signum(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"signum\", col))\n}\n\n// Sign - Computes the signum of the given value.\n//\n// Sign is the Golang equivalent of sign: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sign(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sign\", col))\n}\n\n// Sin - Computes sine of the input column.\n//\n// Sin is the Golang equivalent of sin: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sin(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sin\", col))\n}\n\n// Sinh - Computes hyperbolic sine of the input column.\n//\n// Sinh is the Golang equivalent of sinh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sinh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sinh\", col))\n}\n\n// Sqrt - Computes the square root of the specified float value.\n//\n// Sqrt is the Golang equivalent of sqrt: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sqrt(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sqrt\", col))\n}\n\n// TryAdd - Returns the sum of `left`and `right` and the result is null on overflow.\n// The acceptable input types are the same with the `+` operator.\n//\n// TryAdd is the Golang equivalent of try_add: (left: 'ColumnOrName', right: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryAdd(left column.Column, right column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_add\", left, right))\n}\n\n// TryAvg - Returns the mean calculated from values of a group and the result is null on overflow.\n//\n// TryAvg is the Golang equivalent of try_avg: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryAvg(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_avg\", col))\n}\n\n// TryDivide - Returns `dividend`/`divisor`. It always performs floating point division. Its result is\n// always null if `divisor` is 0.\n//\n// TryDivide is the Golang equivalent of try_divide: (left: 'ColumnOrName', right: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryDivide(left column.Column, right column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_divide\", left, right))\n}\n\n// TryMultiply - Returns `left`*`right` and the result is null on overflow. The acceptable input types are the\n// same with the `*` operator.\n//\n// TryMultiply is the Golang equivalent of try_multiply: (left: 'ColumnOrName', right: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryMultiply(left column.Column, right column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_multiply\", left, right))\n}\n\n// TrySubtract - Returns `left`-`right` and the result is null on overflow. The acceptable input types are the\n// same with the `-` operator.\n//\n// TrySubtract is the Golang equivalent of try_subtract: (left: 'ColumnOrName', right: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TrySubtract(left column.Column, right column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_subtract\", left, right))\n}\n\n// TrySum - Returns the sum calculated from values of a group and the result is null on overflow.\n//\n// TrySum is the Golang equivalent of try_sum: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TrySum(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_sum\", col))\n}\n\n// Tan - Computes tangent of the input column.\n//\n// Tan is the Golang equivalent of tan: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Tan(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"tan\", col))\n}\n\n// Tanh - Computes hyperbolic tangent of the input column.\n//\n// Tanh is the Golang equivalent of tanh: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Tanh(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"tanh\", col))\n}\n\n// ToDegrees -\n//\n// ToDegrees is the Golang equivalent of toDegrees: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToDegrees(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"toDegrees\", col))\n}\n\n// ToRadians -\n//\n// ToRadians is the Golang equivalent of toRadians: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToRadians(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"toRadians\", col))\n}\n\n// Unhex - Inverse of hex. Interprets each pair of characters as a hexadecimal number\n// and converts to the byte representation of number.\n//\n// Unhex is the Golang equivalent of unhex: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Unhex(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unhex\", col))\n}\n\n// ApproxCountDistinct - Aggregate function: returns a new :class:`~pyspark.sql.Column` for approximate distinct count\n// of column `col`.\n//\n// ApproxCountDistinct is the Golang equivalent of approx_count_distinct: (col: 'ColumnOrName', rsd: Optional[float] = None) -> pyspark.sql.connect.column.Column\nfunc ApproxCountDistinct(col column.Column, rsd float64) column.Column {\n\tlit_rsd := Float64Lit(rsd)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"approx_count_distinct\", col, lit_rsd))\n}\n\n// Avg - Aggregate function: returns the average of the values in a group.\n//\n// Avg is the Golang equivalent of avg: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Avg(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"avg\", col))\n}\n\n// CollectList - Aggregate function: returns a list of objects with duplicates.\n//\n// CollectList is the Golang equivalent of collect_list: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CollectList(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"collect_list\", col))\n}\n\n// ArrayAgg - Aggregate function: returns a list of objects with duplicates.\n//\n// ArrayAgg is the Golang equivalent of array_agg: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayAgg(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_agg\", col))\n}\n\n// CollectSet - Aggregate function: returns a set of objects with duplicate elements eliminated.\n//\n// CollectSet is the Golang equivalent of collect_set: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CollectSet(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"collect_set\", col))\n}\n\n// Corr - Returns a new :class:`~pyspark.sql.Column` for the Pearson Correlation Coefficient for\n// “col1“ and “col2“.\n//\n// Corr is the Golang equivalent of corr: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Corr(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"corr\", col1, col2))\n}\n\n// Count - Aggregate function: returns the number of items in a group.\n//\n// Count is the Golang equivalent of count: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Count(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"count\", col))\n}\n\n// CountDistinct - Returns a new :class:`Column` for distinct count of “col“ or “cols“.\n//\n// CountDistinct is the Golang equivalent of count_distinct: (col: 'ColumnOrName', *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CountDistinct(col column.Column, cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, col)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"count_distinct\", vals...))\n}\n\n// CovarPop - Returns a new :class:`~pyspark.sql.Column` for the population covariance of “col1“ and\n// “col2“.\n//\n// CovarPop is the Golang equivalent of covar_pop: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CovarPop(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"covar_pop\", col1, col2))\n}\n\n// CovarSamp - Returns a new :class:`~pyspark.sql.Column` for the sample covariance of “col1“ and\n// “col2“.\n//\n// CovarSamp is the Golang equivalent of covar_samp: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CovarSamp(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"covar_samp\", col1, col2))\n}\n\n// TODO: first: (col: 'ColumnOrName', ignorenulls: bool = False) -> pyspark.sql.connect.column.Column\n\n// Grouping - Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated\n// or not, returns 1 for aggregated or 0 for not aggregated in the result set.\n//\n// Grouping is the Golang equivalent of grouping: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Grouping(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"grouping\", col))\n}\n\n// GroupingId - Aggregate function: returns the level of grouping, equals to\n//\n// (grouping(c1) << (n-1)) + (grouping(c2) << (n-2)) + ... + grouping(cn)\n//\n// GroupingId is the Golang equivalent of grouping_id: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc GroupingId(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"grouping_id\", vals...))\n}\n\n// CountMinSketch - Returns a count-min sketch of a column with the given esp, confidence and seed.\n// The result is an array of bytes, which can be deserialized to a `CountMinSketch` before usage.\n// Count-min sketch is a probabilistic data structure used for cardinality estimation\n// using sub-linear space.\n//\n// CountMinSketch is the Golang equivalent of count_min_sketch: (col: 'ColumnOrName', eps: 'ColumnOrName', confidence: 'ColumnOrName', seed: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CountMinSketch(col column.Column, eps column.Column, confidence column.Column, seed column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"count_min_sketch\", col, eps, confidence, seed))\n}\n\n// Kurtosis - Aggregate function: returns the kurtosis of the values in a group.\n//\n// Kurtosis is the Golang equivalent of kurtosis: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Kurtosis(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"kurtosis\", col))\n}\n\n// TODO: last: (col: 'ColumnOrName', ignorenulls: bool = False) -> pyspark.sql.connect.column.Column\n\n// Max - Aggregate function: returns the maximum value of the expression in a group.\n//\n// Max is the Golang equivalent of max: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Max(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"max\", col))\n}\n\n// MaxBy - Returns the value associated with the maximum value of ord.\n//\n// MaxBy is the Golang equivalent of max_by: (col: 'ColumnOrName', ord: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MaxBy(col column.Column, ord column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"max_by\", col, ord))\n}\n\n// Mean - Aggregate function: returns the average of the values in a group.\n// An alias of :func:`avg`.\n//\n// Mean is the Golang equivalent of mean: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Mean(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"mean\", col))\n}\n\n// Median - Returns the median of the values in a group.\n//\n// Median is the Golang equivalent of median: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Median(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"median\", col))\n}\n\n// Min - Aggregate function: returns the minimum value of the expression in a group.\n//\n// Min is the Golang equivalent of min: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Min(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"min\", col))\n}\n\n// MinBy - Returns the value associated with the minimum value of ord.\n//\n// MinBy is the Golang equivalent of min_by: (col: 'ColumnOrName', ord: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MinBy(col column.Column, ord column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"min_by\", col, ord))\n}\n\n// Mode - Returns the most frequent value in a group.\n//\n// Mode is the Golang equivalent of mode: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Mode(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"mode\", col))\n}\n\n// TODO: percentile: (col: 'ColumnOrName', percentage: Union[pyspark.sql.connect.column.Column, float, List[float], Tuple[float]], frequency: Union[pyspark.sql.connect.column.Column, int] = 1) -> pyspark.sql.connect.column.Column\n\n// TODO: percentile_approx: (col: 'ColumnOrName', percentage: Union[pyspark.sql.connect.column.Column, float, List[float], Tuple[float]], accuracy: Union[pyspark.sql.connect.column.Column, float] = 10000) -> pyspark.sql.connect.column.Column\n\n// TODO: approx_percentile: (col: 'ColumnOrName', percentage: Union[pyspark.sql.connect.column.Column, float, List[float], Tuple[float]], accuracy: Union[pyspark.sql.connect.column.Column, float] = 10000) -> pyspark.sql.connect.column.Column\n\n// Product - Aggregate function: returns the product of the values in a group.\n//\n// Product is the Golang equivalent of product: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Product(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"product\", col))\n}\n\n// Skewness - Aggregate function: returns the skewness of the values in a group.\n//\n// Skewness is the Golang equivalent of skewness: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Skewness(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"skewness\", col))\n}\n\n// Stddev - Aggregate function: alias for stddev_samp.\n//\n// Stddev is the Golang equivalent of stddev: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Stddev(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"stddev\", col))\n}\n\n// Std - Aggregate function: alias for stddev_samp.\n//\n// Std is the Golang equivalent of std: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Std(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"std\", col))\n}\n\n// StddevSamp - Aggregate function: returns the unbiased sample standard deviation of\n// the expression in a group.\n//\n// StddevSamp is the Golang equivalent of stddev_samp: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc StddevSamp(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"stddev_samp\", col))\n}\n\n// StddevPop - Aggregate function: returns population standard deviation of\n// the expression in a group.\n//\n// StddevPop is the Golang equivalent of stddev_pop: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc StddevPop(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"stddev_pop\", col))\n}\n\n// Sum - Aggregate function: returns the sum of all values in the expression.\n//\n// Sum is the Golang equivalent of sum: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sum(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sum\", col))\n}\n\n// SumDistinct - Aggregate function: returns the sum of distinct values in the expression.\n//\n// SumDistinct is the Golang equivalent of sum_distinct: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc SumDistinct(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sum_distinct\", col))\n}\n\n// VarPop - Aggregate function: returns the population variance of the values in a group.\n//\n// VarPop is the Golang equivalent of var_pop: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc VarPop(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"var_pop\", col))\n}\n\n// RegrAvgx - Aggregate function: returns the average of the independent variable for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrAvgx is the Golang equivalent of regr_avgx: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrAvgx(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_avgx\", y, x))\n}\n\n// RegrAvgy - Aggregate function: returns the average of the dependent variable for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrAvgy is the Golang equivalent of regr_avgy: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrAvgy(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_avgy\", y, x))\n}\n\n// RegrCount - Aggregate function: returns the number of non-null number pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrCount is the Golang equivalent of regr_count: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrCount(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_count\", y, x))\n}\n\n// RegrIntercept - Aggregate function: returns the intercept of the univariate linear regression line\n// for non-null pairs in a group, where `y` is the dependent variable and\n// `x` is the independent variable.\n//\n// RegrIntercept is the Golang equivalent of regr_intercept: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrIntercept(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_intercept\", y, x))\n}\n\n// RegrR2 - Aggregate function: returns the coefficient of determination for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrR2 is the Golang equivalent of regr_r2: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrR2(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_r2\", y, x))\n}\n\n// RegrSlope - Aggregate function: returns the slope of the linear regression line for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrSlope is the Golang equivalent of regr_slope: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrSlope(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_slope\", y, x))\n}\n\n// RegrSxx - Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(x) for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrSxx is the Golang equivalent of regr_sxx: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrSxx(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_sxx\", y, x))\n}\n\n// RegrSxy - Aggregate function: returns REGR_COUNT(y, x) * COVAR_POP(y, x) for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrSxy is the Golang equivalent of regr_sxy: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrSxy(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_sxy\", y, x))\n}\n\n// RegrSyy - Aggregate function: returns REGR_COUNT(y, x) * VAR_POP(y) for non-null pairs\n// in a group, where `y` is the dependent variable and `x` is the independent variable.\n//\n// RegrSyy is the Golang equivalent of regr_syy: (y: 'ColumnOrName', x: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegrSyy(y column.Column, x column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regr_syy\", y, x))\n}\n\n// VarSamp - Aggregate function: returns the unbiased sample variance of\n// the values in a group.\n//\n// VarSamp is the Golang equivalent of var_samp: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc VarSamp(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"var_samp\", col))\n}\n\n// Variance - Aggregate function: alias for var_samp\n//\n// Variance is the Golang equivalent of variance: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Variance(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"variance\", col))\n}\n\n// Every - Aggregate function: returns true if all values of `col` are true.\n//\n// Every is the Golang equivalent of every: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Every(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"every\", col))\n}\n\n// BoolAnd - Aggregate function: returns true if all values of `col` are true.\n//\n// BoolAnd is the Golang equivalent of bool_and: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BoolAnd(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bool_and\", col))\n}\n\n// Some - Aggregate function: returns true if at least one value of `col` is true.\n//\n// Some is the Golang equivalent of some: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Some(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"some\", col))\n}\n\n// BoolOr - Aggregate function: returns true if at least one value of `col` is true.\n//\n// BoolOr is the Golang equivalent of bool_or: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BoolOr(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bool_or\", col))\n}\n\n// BitAnd - Aggregate function: returns the bitwise AND of all non-null input values, or null if none.\n//\n// BitAnd is the Golang equivalent of bit_and: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitAnd(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_and\", col))\n}\n\n// BitOr - Aggregate function: returns the bitwise OR of all non-null input values, or null if none.\n//\n// BitOr is the Golang equivalent of bit_or: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitOr(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_or\", col))\n}\n\n// BitXor - Aggregate function: returns the bitwise XOR of all non-null input values, or null if none.\n//\n// BitXor is the Golang equivalent of bit_xor: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitXor(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_xor\", col))\n}\n\n// CumeDist - Window function: returns the cumulative distribution of values within a window partition,\n// i.e. the fraction of rows that are below the current row.\n//\n// CumeDist is the Golang equivalent of cume_dist: () -> pyspark.sql.connect.column.Column\nfunc CumeDist() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cume_dist\"))\n}\n\n// DenseRank - Window function: returns the rank of rows within a window partition, without any gaps.\n//\n// The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking\n// sequence when there are ties. That is, if you were ranking a competition using dense_rank\n// and had three people tie for second place, you would say that all three were in second\n// place and that the next person came in third. Rank would give me sequential numbers, making\n// the person that came in third place (after the ties) would register as coming in fifth.\n//\n// This is equivalent to the DENSE_RANK function in SQL.\n//\n// DenseRank is the Golang equivalent of dense_rank: () -> pyspark.sql.connect.column.Column\nfunc DenseRank() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"dense_rank\"))\n}\n\n// TODO: lag: (col: 'ColumnOrName', offset: int = 1, default: Optional[Any] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: lead: (col: 'ColumnOrName', offset: int = 1, default: Optional[Any] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: nth_value: (col: 'ColumnOrName', offset: int, ignoreNulls: Optional[bool] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: any_value: (col: 'ColumnOrName', ignoreNulls: Union[bool, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: first_value: (col: 'ColumnOrName', ignoreNulls: Union[bool, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: last_value: (col: 'ColumnOrName', ignoreNulls: Union[bool, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// CountIf - Returns the number of `TRUE` values for the `col`.\n//\n// CountIf is the Golang equivalent of count_if: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CountIf(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"count_if\", col))\n}\n\n// HistogramNumeric - Computes a histogram on numeric 'col' using nb bins.\n// The return value is an array of (x,y) pairs representing the centers of the\n// histogram's bins. As the value of 'nb' is increased, the histogram approximation\n// gets finer-grained, but may yield artifacts around outliers. In practice, 20-40\n// histogram bins appear to work well, with more bins being required for skewed or\n// smaller datasets. Note that this function creates a histogram with non-uniform\n// bin widths. It offers no guarantees in terms of the mean-squared-error of the\n// histogram, but in practice is comparable to the histograms produced by the R/S-Plus\n// statistical computing packages. Note: the output type of the 'x' field in the return value is\n// propagated from the input value consumed in the aggregate function.\n//\n// HistogramNumeric is the Golang equivalent of histogram_numeric: (col: 'ColumnOrName', nBins: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc HistogramNumeric(col column.Column, nBins column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"histogram_numeric\", col, nBins))\n}\n\n// Ntile - Window function: returns the ntile group id (from 1 to `n` inclusive)\n// in an ordered window partition. For example, if `n` is 4, the first\n// quarter of the rows will get value 1, the second quarter will get 2,\n// the third quarter will get 3, and the last quarter will get 4.\n//\n// This is equivalent to the NTILE function in SQL.\n//\n// Ntile is the Golang equivalent of ntile: (n: int) -> pyspark.sql.connect.column.Column\nfunc Ntile(n int64) column.Column {\n\tlit_n := Int64Lit(n)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ntile\", lit_n))\n}\n\n// PercentRank - Window function: returns the relative rank (i.e. percentile) of rows within a window partition.\n//\n// PercentRank is the Golang equivalent of percent_rank: () -> pyspark.sql.connect.column.Column\nfunc PercentRank() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"percent_rank\"))\n}\n\n// Rank - Window function: returns the rank of rows within a window partition.\n//\n// The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking\n// sequence when there are ties. That is, if you were ranking a competition using dense_rank\n// and had three people tie for second place, you would say that all three were in second\n// place and that the next person came in third. Rank would give me sequential numbers, making\n// the person that came in third place (after the ties) would register as coming in fifth.\n//\n// This is equivalent to the RANK function in SQL.\n//\n// Rank is the Golang equivalent of rank: () -> pyspark.sql.connect.column.Column\nfunc Rank() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rank\"))\n}\n\n// RowNumber - Window function: returns a sequential number starting at 1 within a window partition.\n//\n// RowNumber is the Golang equivalent of row_number: () -> pyspark.sql.connect.column.Column\nfunc RowNumber() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"row_number\"))\n}\n\n// TODO: aggregate: (col: 'ColumnOrName', initialValue: 'ColumnOrName', merge: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column], finish: Optional[Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: reduce: (col: 'ColumnOrName', initialValue: 'ColumnOrName', merge: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column], finish: Optional[Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]] = None) -> pyspark.sql.connect.column.Column\n\n// Array - Creates a new array column.\n//\n// Array is the Golang equivalent of array: (*cols: Union[ForwardRef('ColumnOrName'), List[ForwardRef('ColumnOrName')], Tuple[ForwardRef('ColumnOrName'), ...]]) -> pyspark.sql.connect.column.Column\nfunc Array(cols column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array\", cols))\n}\n\n// TODO: array_append: (col: 'ColumnOrName', value: Any) -> pyspark.sql.connect.column.Column\n\n// TODO: array_contains: (col: 'ColumnOrName', value: Any) -> pyspark.sql.connect.column.Column\n\n// ArrayDistinct - Collection function: removes duplicate values from the array.\n//\n// ArrayDistinct is the Golang equivalent of array_distinct: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayDistinct(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_distinct\", col))\n}\n\n// ArrayExcept - Collection function: returns an array of the elements in col1 but not in col2,\n// without duplicates.\n//\n// ArrayExcept is the Golang equivalent of array_except: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayExcept(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_except\", col1, col2))\n}\n\n// TODO: array_insert: (arr: 'ColumnOrName', pos: Union[ForwardRef('ColumnOrName'), int], value: Any) -> pyspark.sql.connect.column.Column\n\n// ArrayIntersect - Collection function: returns an array of the elements in the intersection of col1 and col2,\n// without duplicates.\n//\n// ArrayIntersect is the Golang equivalent of array_intersect: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayIntersect(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_intersect\", col1, col2))\n}\n\n// ArrayCompact - Collection function: removes null values from the array.\n//\n// ArrayCompact is the Golang equivalent of array_compact: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayCompact(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_compact\", col))\n}\n\n// ArrayJoin - Concatenates the elements of `column` using the `delimiter`. Null values are replaced with\n// `null_replacement` if set, otherwise they are ignored.\n//\n// ArrayJoin is the Golang equivalent of array_join: (col: 'ColumnOrName', delimiter: str, null_replacement: Optional[str] = None) -> pyspark.sql.connect.column.Column\nfunc ArrayJoin(col column.Column, delimiter string, null_replacement string) column.Column {\n\tlit_delimiter := StringLit(delimiter)\n\tlit_null_replacement := StringLit(null_replacement)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_join\", col, lit_delimiter, lit_null_replacement))\n}\n\n// ArrayMax - Collection function: returns the maximum value of the array.\n//\n// ArrayMax is the Golang equivalent of array_max: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayMax(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_max\", col))\n}\n\n// ArrayMin - Collection function: returns the minimum value of the array.\n//\n// ArrayMin is the Golang equivalent of array_min: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayMin(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_min\", col))\n}\n\n// ArraySize - Returns the total number of elements in the array. The function returns null for null input.\n//\n// ArraySize is the Golang equivalent of array_size: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArraySize(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_size\", col))\n}\n\n// Cardinality - Collection function: returns the length of the array or map stored in the column.\n//\n// Cardinality is the Golang equivalent of cardinality: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Cardinality(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"cardinality\", col))\n}\n\n// TODO: array_position: (col: 'ColumnOrName', value: Any) -> pyspark.sql.connect.column.Column\n\n// TODO: array_prepend: (col: 'ColumnOrName', value: Any) -> pyspark.sql.connect.column.Column\n\n// TODO: array_remove: (col: 'ColumnOrName', element: Any) -> pyspark.sql.connect.column.Column\n\n// ArrayRepeat - Collection function: creates an array containing a column repeated count times.\n//\n// ArrayRepeat is the Golang equivalent of array_repeat: (col: 'ColumnOrName', count: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc ArrayRepeat(col column.Column, count column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_repeat\", col, count))\n}\n\n// TODO: array_sort: (col: 'ColumnOrName', comparator: Optional[Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]] = None) -> pyspark.sql.connect.column.Column\n\n// ArrayUnion - Collection function: returns an array of the elements in the union of col1 and col2,\n// without duplicates.\n//\n// ArrayUnion is the Golang equivalent of array_union: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArrayUnion(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"array_union\", col1, col2))\n}\n\n// ArraysOverlap - Collection function: returns true if the arrays contain any common non-null element; if not,\n// returns null if both the arrays are non-empty and any of them contains a null element; returns\n// false otherwise.\n//\n// ArraysOverlap is the Golang equivalent of arrays_overlap: (a1: 'ColumnOrName', a2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArraysOverlap(a1 column.Column, a2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"arrays_overlap\", a1, a2))\n}\n\n// ArraysZip - Collection function: Returns a merged array of structs in which the N-th struct contains all\n// N-th values of input arrays. If one of the arrays is shorter than others then\n// resulting struct type value will be a `null` for missing elements.\n//\n// ArraysZip is the Golang equivalent of arrays_zip: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ArraysZip(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"arrays_zip\", vals...))\n}\n\n// Concat - Concatenates multiple input columns together into a single column.\n// The function works with strings, numeric, binary and compatible array columns.\n//\n// Concat is the Golang equivalent of concat: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Concat(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"concat\", vals...))\n}\n\n// CreateMap - Creates a new map column.\n//\n// CreateMap is the Golang equivalent of create_map: (*cols: Union[ForwardRef('ColumnOrName'), List[ForwardRef('ColumnOrName')], Tuple[ForwardRef('ColumnOrName'), ...]]) -> pyspark.sql.connect.column.Column\nfunc CreateMap(cols column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"create_map\", cols))\n}\n\n// TODO: element_at: (col: 'ColumnOrName', extraction: Any) -> pyspark.sql.connect.column.Column\n\n// TryElementAt - (array, index) - Returns element of array at given (1-based) index. If Index is 0, Spark will\n// throw an error. If index < 0, accesses elements from the last to the first. The function\n// always returns NULL if the index exceeds the length of the array.\n//\n// (map, key) - Returns value for given key. The function always returns NULL if the key is not\n// contained in the map.\n//\n// TryElementAt is the Golang equivalent of try_element_at: (col: 'ColumnOrName', extraction: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryElementAt(col column.Column, extraction column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_element_at\", col, extraction))\n}\n\n// TODO: exists: (col: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// Explode - Returns a new row for each element in the given array or map.\n// Uses the default column name `col` for elements in the array and\n// `key` and `value` for elements in the map unless specified otherwise.\n//\n// Explode is the Golang equivalent of explode: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Explode(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"explode\", col))\n}\n\n// ExplodeOuter - Returns a new row for each element in the given array or map.\n// Unlike explode, if the array/map is null or empty then null is produced.\n// Uses the default column name `col` for elements in the array and\n// `key` and `value` for elements in the map unless specified otherwise.\n//\n// ExplodeOuter is the Golang equivalent of explode_outer: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ExplodeOuter(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"explode_outer\", col))\n}\n\n// TODO: filter: (col: 'ColumnOrName', f: Union[Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column], Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]]) -> pyspark.sql.connect.column.Column\n\n// Flatten - Collection function: creates a single array from an array of arrays.\n// If a structure of nested arrays is deeper than two levels,\n// only one level of nesting is removed.\n//\n// Flatten is the Golang equivalent of flatten: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Flatten(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"flatten\", col))\n}\n\n// TODO: forall: (col: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// TODO: from_csv: (col: 'ColumnOrName', schema: Union[pyspark.sql.connect.column.Column, str], options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: from_json: (col: 'ColumnOrName', schema: Union[pyspark.sql.types.ArrayType, pyspark.sql.types.StructType, pyspark.sql.connect.column.Column, str], options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// Get - Collection function: Returns element of array at given (0-based) index.\n// If the index points outside of the array boundaries, then this function\n// returns NULL.\n//\n// Get is the Golang equivalent of get: (col: 'ColumnOrName', index: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc Get(col column.Column, index column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"get\", col, index))\n}\n\n// GetJsonObject - Extracts json object from a json string based on json `path` specified, and returns json string\n// of the extracted json object. It will return null if the input json string is invalid.\n//\n// GetJsonObject is the Golang equivalent of get_json_object: (col: 'ColumnOrName', path: str) -> pyspark.sql.connect.column.Column\nfunc GetJsonObject(col column.Column, path string) column.Column {\n\tlit_path := StringLit(path)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"get_json_object\", col, lit_path))\n}\n\n// JsonArrayLength - Returns the number of elements in the outermost JSON array. `NULL` is returned in case of\n// any other valid JSON string, `NULL` or an invalid JSON.\n//\n// JsonArrayLength is the Golang equivalent of json_array_length: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc JsonArrayLength(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"json_array_length\", col))\n}\n\n// JsonObjectKeys - Returns all the keys of the outermost JSON object as an array. If a valid JSON object is\n// given, all the keys of the outermost object will be returned as an array. If it is any\n// other valid JSON string, an invalid JSON string or an empty string, the function returns null.\n//\n// JsonObjectKeys is the Golang equivalent of json_object_keys: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc JsonObjectKeys(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"json_object_keys\", col))\n}\n\n// Inline - Explodes an array of structs into a table.\n//\n// Inline is the Golang equivalent of inline: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Inline(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"inline\", col))\n}\n\n// InlineOuter - Explodes an array of structs into a table.\n// Unlike inline, if the array is null or empty then null is produced for each nested column.\n//\n// InlineOuter is the Golang equivalent of inline_outer: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc InlineOuter(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"inline_outer\", col))\n}\n\n// JsonTuple - Creates a new row for a json column according to the given field names.\n//\n// JsonTuple is the Golang equivalent of json_tuple: (col: 'ColumnOrName', *fields: str) -> pyspark.sql.connect.column.Column\nfunc JsonTuple(col column.Column, fields string) column.Column {\n\tlit_fields := StringLit(fields)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"json_tuple\", col, lit_fields))\n}\n\n// MapConcat - Returns the union of all the given maps.\n//\n// MapConcat is the Golang equivalent of map_concat: (*cols: Union[ForwardRef('ColumnOrName'), List[ForwardRef('ColumnOrName')], Tuple[ForwardRef('ColumnOrName'), ...]]) -> pyspark.sql.connect.column.Column\nfunc MapConcat(cols column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_concat\", cols))\n}\n\n// TODO: map_contains_key: (col: 'ColumnOrName', value: Any) -> pyspark.sql.connect.column.Column\n\n// MapEntries - Collection function: Returns an unordered array of all entries in the given map.\n//\n// MapEntries is the Golang equivalent of map_entries: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MapEntries(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_entries\", col))\n}\n\n// TODO: map_filter: (col: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// MapFromArrays - Creates a new map from two arrays.\n//\n// MapFromArrays is the Golang equivalent of map_from_arrays: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MapFromArrays(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_from_arrays\", col1, col2))\n}\n\n// MapFromEntries - Collection function: Converts an array of entries (key value struct types) to a map\n// of values.\n//\n// MapFromEntries is the Golang equivalent of map_from_entries: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MapFromEntries(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_from_entries\", col))\n}\n\n// MapKeys - Collection function: Returns an unordered array containing the keys of the map.\n//\n// MapKeys is the Golang equivalent of map_keys: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MapKeys(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_keys\", col))\n}\n\n// MapValues - Collection function: Returns an unordered array containing the values of the map.\n//\n// MapValues is the Golang equivalent of map_values: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MapValues(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"map_values\", col))\n}\n\n// TODO: map_zip_with: (col1: 'ColumnOrName', col2: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// StrToMap - Creates a map after splitting the text into key/value pairs using delimiters.\n// Both `pairDelim` and `keyValueDelim` are treated as regular expressions.\n//\n// StrToMap is the Golang equivalent of str_to_map: (text: 'ColumnOrName', pairDelim: Optional[ForwardRef('ColumnOrName')] = None, keyValueDelim: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc StrToMap(text column.Column, pairDelim column.Column, keyValueDelim column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"str_to_map\", text, pairDelim, keyValueDelim))\n}\n\n// Posexplode - Returns a new row for each element with position in the given array or map.\n// Uses the default column name `pos` for position, and `col` for elements in the\n// array and `key` and `value` for elements in the map unless specified otherwise.\n//\n// Posexplode is the Golang equivalent of posexplode: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Posexplode(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"posexplode\", col))\n}\n\n// PosexplodeOuter - Returns a new row for each element with position in the given array or map.\n// Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced.\n// Uses the default column name `pos` for position, and `col` for elements in the\n// array and `key` and `value` for elements in the map unless specified otherwise.\n//\n// PosexplodeOuter is the Golang equivalent of posexplode_outer: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc PosexplodeOuter(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"posexplode_outer\", col))\n}\n\n// Reverse - Collection function: returns a reversed string or an array with reverse order of elements.\n//\n// Reverse is the Golang equivalent of reverse: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Reverse(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"reverse\", col))\n}\n\n// Sequence - Generate a sequence of integers from `start` to `stop`, incrementing by `step`.\n// If `step` is not set, incrementing by 1 if `start` is less than or equal to `stop`,\n// otherwise -1.\n//\n// Sequence is the Golang equivalent of sequence: (start: 'ColumnOrName', stop: 'ColumnOrName', step: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Sequence(start column.Column, stop column.Column, step column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sequence\", start, stop, step))\n}\n\n// TODO: schema_of_csv: (csv: 'ColumnOrName', options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: schema_of_json: (json: 'ColumnOrName', options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// Shuffle - Collection function: Generates a random permutation of the given array.\n//\n// Shuffle is the Golang equivalent of shuffle: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Shuffle(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"shuffle\", col))\n}\n\n// Size - Collection function: returns the length of the array or map stored in the column.\n//\n// Size is the Golang equivalent of size: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Size(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"size\", col))\n}\n\n// Slice - Collection function: returns an array containing all the elements in `x` from index `start`\n// (array indices start at 1, or from the end if `start` is negative) with the specified `length`.\n//\n// Slice is the Golang equivalent of slice: (col: 'ColumnOrName', start: Union[ForwardRef('ColumnOrName'), int], length: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc Slice(col column.Column, start column.Column, length column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"slice\", col, start, length))\n}\n\n// TODO: sort_array: (col: 'ColumnOrName', asc: bool = True) -> pyspark.sql.connect.column.Column\n\n// Struct - Creates a new struct column.\n//\n// Struct is the Golang equivalent of struct: (*cols: Union[ForwardRef('ColumnOrName'), List[ForwardRef('ColumnOrName')], Tuple[ForwardRef('ColumnOrName'), ...]]) -> pyspark.sql.connect.column.Column\nfunc Struct(cols column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"struct\", cols))\n}\n\n// NamedStruct - Creates a struct with the given field names and values.\n//\n// NamedStruct is the Golang equivalent of named_struct: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc NamedStruct(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"named_struct\", vals...))\n}\n\n// TODO: to_csv: (col: 'ColumnOrName', options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: to_json: (col: 'ColumnOrName', options: Optional[Dict[str, str]] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: transform: (col: 'ColumnOrName', f: Union[Callable[[pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column], Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]]) -> pyspark.sql.connect.column.Column\n\n// TODO: transform_keys: (col: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// TODO: transform_values: (col: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// TODO: zip_with: (left: 'ColumnOrName', right: 'ColumnOrName', f: Callable[[pyspark.sql.connect.column.Column, pyspark.sql.connect.column.Column], pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// Upper - Converts a string expression to upper case.\n//\n// Upper is the Golang equivalent of upper: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Upper(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"upper\", col))\n}\n\n// Lower - Converts a string expression to lower case.\n//\n// Lower is the Golang equivalent of lower: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Lower(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"lower\", col))\n}\n\n// Ascii - Computes the numeric value of the first character of the string column.\n//\n// Ascii is the Golang equivalent of ascii: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ascii(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ascii\", col))\n}\n\n// Base64 - Computes the BASE64 encoding of a binary column and returns it as a string column.\n//\n// Base64 is the Golang equivalent of base64: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Base64(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"base64\", col))\n}\n\n// Unbase64 - Decodes a BASE64 encoded string column and returns it as a binary column.\n//\n// Unbase64 is the Golang equivalent of unbase64: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Unbase64(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unbase64\", col))\n}\n\n// Ltrim - Trim the spaces from left end for the specified string value.\n//\n// Ltrim is the Golang equivalent of ltrim: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ltrim(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ltrim\", col))\n}\n\n// Rtrim - Trim the spaces from right end for the specified string value.\n//\n// Rtrim is the Golang equivalent of rtrim: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Rtrim(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rtrim\", col))\n}\n\n// Trim - Trim the spaces from both ends for the specified string column.\n//\n// Trim is the Golang equivalent of trim: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Trim(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"trim\", col))\n}\n\n// ConcatWs - Concatenates multiple input string columns together into a single string column,\n// using the given separator.\n//\n// ConcatWs is the Golang equivalent of concat_ws: (sep: str, *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ConcatWs(sep string, cols ...column.Column) column.Column {\n\tlit_sep := StringLit(sep)\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, lit_sep)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"concat_ws\", vals...))\n}\n\n// Decode - Computes the first argument into a string from a binary using the provided character set\n// (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').\n//\n// Decode is the Golang equivalent of decode: (col: 'ColumnOrName', charset: str) -> pyspark.sql.connect.column.Column\nfunc Decode(col column.Column, charset string) column.Column {\n\tlit_charset := StringLit(charset)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"decode\", col, lit_charset))\n}\n\n// Encode - Computes the first argument into a binary from a string using the provided character set\n// (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').\n//\n// Encode is the Golang equivalent of encode: (col: 'ColumnOrName', charset: str) -> pyspark.sql.connect.column.Column\nfunc Encode(col column.Column, charset string) column.Column {\n\tlit_charset := StringLit(charset)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"encode\", col, lit_charset))\n}\n\n// FormatNumber - Formats the number X to a format like '#,--#,--#.--', rounded to d decimal places\n// with HALF_EVEN round mode, and returns the result as a string.\n//\n// FormatNumber is the Golang equivalent of format_number: (col: 'ColumnOrName', d: int) -> pyspark.sql.connect.column.Column\nfunc FormatNumber(col column.Column, d int64) column.Column {\n\tlit_d := Int64Lit(d)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"format_number\", col, lit_d))\n}\n\n// FormatString - Formats the arguments in printf-style and returns the result as a string column.\n//\n// FormatString is the Golang equivalent of format_string: (format: str, *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc FormatString(format string, cols ...column.Column) column.Column {\n\tlit_format := StringLit(format)\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, lit_format)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"format_string\", vals...))\n}\n\n// Instr - Locate the position of the first occurrence of substr column in the given string.\n// Returns null if either of the arguments are null.\n//\n// Instr is the Golang equivalent of instr: (str: 'ColumnOrName', substr: str) -> pyspark.sql.connect.column.Column\nfunc Instr(str column.Column, substr string) column.Column {\n\tlit_substr := StringLit(substr)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"instr\", str, lit_substr))\n}\n\n// Overlay - Overlay the specified portion of `src` with `replace`,\n// starting from byte position `pos` of `src` and proceeding for `len` bytes.\n//\n// Overlay is the Golang equivalent of overlay: (src: 'ColumnOrName', replace: 'ColumnOrName', pos: Union[ForwardRef('ColumnOrName'), int], len: Union[ForwardRef('ColumnOrName'), int] = -1) -> pyspark.sql.connect.column.Column\nfunc Overlay(src column.Column, replace column.Column, pos column.Column, len column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"overlay\", src, replace, pos, len))\n}\n\n// Sentences - Splits a string into arrays of sentences, where each sentence is an array of words.\n// The 'language' and 'country' arguments are optional, and if omitted, the default locale is used.\n//\n// Sentences is the Golang equivalent of sentences: (string: 'ColumnOrName', language: Optional[ForwardRef('ColumnOrName')] = None, country: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Sentences(string column.Column, language column.Column, country column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sentences\", string, language, country))\n}\n\n// Substring - Substring starts at `pos` and is of length `len` when str is String type or\n// returns the slice of byte array that starts at `pos` in byte and is of length `len`\n// when str is Binary type.\n//\n// Substring is the Golang equivalent of substring: (str: 'ColumnOrName', pos: int, len: int) -> pyspark.sql.connect.column.Column\nfunc Substring(str column.Column, pos int64, len int64) column.Column {\n\tlit_pos := Int64Lit(pos)\n\tlit_len := Int64Lit(len)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"substring\", str, lit_pos, lit_len))\n}\n\n// SubstringIndex - Returns the substring from string str before count occurrences of the delimiter delim.\n// If count is positive, everything the left of the final delimiter (counting from left) is\n// returned. If count is negative, every to the right of the final delimiter (counting from the\n// right) is returned. substring_index performs a case-sensitive match when searching for delim.\n//\n// SubstringIndex is the Golang equivalent of substring_index: (str: 'ColumnOrName', delim: str, count: int) -> pyspark.sql.connect.column.Column\nfunc SubstringIndex(str column.Column, delim string, count int64) column.Column {\n\tlit_delim := StringLit(delim)\n\tlit_count := Int64Lit(count)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"substring_index\", str, lit_delim, lit_count))\n}\n\n// Levenshtein - Computes the Levenshtein distance of the two given strings.\n//\n// Levenshtein is the Golang equivalent of levenshtein: (left: 'ColumnOrName', right: 'ColumnOrName', threshold: Optional[int] = None) -> pyspark.sql.connect.column.Column\nfunc Levenshtein(left column.Column, right column.Column, threshold int64) column.Column {\n\tlit_threshold := Int64Lit(threshold)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"levenshtein\", left, right, lit_threshold))\n}\n\n// Locate - Locate the position of the first occurrence of substr in a string column, after position pos.\n//\n// Locate is the Golang equivalent of locate: (substr: str, str: 'ColumnOrName', pos: int = 1) -> pyspark.sql.connect.column.Column\nfunc Locate(substr string, str column.Column, pos int64) column.Column {\n\tlit_substr := StringLit(substr)\n\tlit_pos := Int64Lit(pos)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"locate\", lit_substr, str, lit_pos))\n}\n\n// Lpad - Left-pad the string column to width `len` with `pad`.\n//\n// Lpad is the Golang equivalent of lpad: (col: 'ColumnOrName', len: int, pad: str) -> pyspark.sql.connect.column.Column\nfunc Lpad(col column.Column, len int64, pad string) column.Column {\n\tlit_len := Int64Lit(len)\n\tlit_pad := StringLit(pad)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"lpad\", col, lit_len, lit_pad))\n}\n\n// Rpad - Right-pad the string column to width `len` with `pad`.\n//\n// Rpad is the Golang equivalent of rpad: (col: 'ColumnOrName', len: int, pad: str) -> pyspark.sql.connect.column.Column\nfunc Rpad(col column.Column, len int64, pad string) column.Column {\n\tlit_len := Int64Lit(len)\n\tlit_pad := StringLit(pad)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rpad\", col, lit_len, lit_pad))\n}\n\n// Repeat - Repeats a string column n times, and returns it as a new string column.\n//\n// Repeat is the Golang equivalent of repeat: (col: 'ColumnOrName', n: int) -> pyspark.sql.connect.column.Column\nfunc Repeat(col column.Column, n int64) column.Column {\n\tlit_n := Int64Lit(n)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"repeat\", col, lit_n))\n}\n\n// Split - Splits str around matches of the given pattern.\n//\n// Split is the Golang equivalent of split: (str: 'ColumnOrName', pattern: str, limit: int = -1) -> pyspark.sql.connect.column.Column\nfunc Split(str column.Column, pattern string, limit int64) column.Column {\n\tlit_pattern := StringLit(pattern)\n\tlit_limit := Int64Lit(limit)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"split\", str, lit_pattern, lit_limit))\n}\n\n// Rlike - Returns true if `str` matches the Java regex `regexp`, or false otherwise.\n//\n// Rlike is the Golang equivalent of rlike: (str: 'ColumnOrName', regexp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Rlike(str column.Column, regexp column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"rlike\", str, regexp))\n}\n\n// Regexp - Returns true if `str` matches the Java regex `regexp`, or false otherwise.\n//\n// Regexp is the Golang equivalent of regexp: (str: 'ColumnOrName', regexp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Regexp(str column.Column, regexp column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regexp\", str, regexp))\n}\n\n// RegexpLike - Returns true if `str` matches the Java regex `regexp`, or false otherwise.\n//\n// RegexpLike is the Golang equivalent of regexp_like: (str: 'ColumnOrName', regexp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegexpLike(str column.Column, regexp column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regexp_like\", str, regexp))\n}\n\n// RegexpCount - Returns a count of the number of times that the Java regex pattern `regexp` is matched\n// in the string `str`.\n//\n// RegexpCount is the Golang equivalent of regexp_count: (str: 'ColumnOrName', regexp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegexpCount(str column.Column, regexp column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regexp_count\", str, regexp))\n}\n\n// RegexpExtract - Extract a specific group matched by the Java regex `regexp`, from the specified string column.\n// If the regex did not match, or the specified group did not match, an empty string is returned.\n//\n// RegexpExtract is the Golang equivalent of regexp_extract: (str: 'ColumnOrName', pattern: str, idx: int) -> pyspark.sql.connect.column.Column\nfunc RegexpExtract(str column.Column, pattern string, idx int64) column.Column {\n\tlit_pattern := StringLit(pattern)\n\tlit_idx := Int64Lit(idx)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regexp_extract\", str, lit_pattern, lit_idx))\n}\n\n// TODO: regexp_extract_all: (str: 'ColumnOrName', regexp: 'ColumnOrName', idx: Union[int, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: regexp_replace: (string: 'ColumnOrName', pattern: Union[str, pyspark.sql.connect.column.Column], replacement: Union[str, pyspark.sql.connect.column.Column]) -> pyspark.sql.connect.column.Column\n\n// RegexpSubstr - Returns the substring that matches the Java regex `regexp` within the string `str`.\n// If the regular expression is not found, the result is null.\n//\n// RegexpSubstr is the Golang equivalent of regexp_substr: (str: 'ColumnOrName', regexp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc RegexpSubstr(str column.Column, regexp column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"regexp_substr\", str, regexp))\n}\n\n// TODO: regexp_instr: (str: 'ColumnOrName', regexp: 'ColumnOrName', idx: Union[int, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// Initcap - Translate the first letter of each word to upper case in the sentence.\n//\n// Initcap is the Golang equivalent of initcap: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Initcap(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"initcap\", col))\n}\n\n// Soundex - Returns the SoundEx encoding for a string\n//\n// Soundex is the Golang equivalent of soundex: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Soundex(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"soundex\", col))\n}\n\n// Length - Computes the character length of string data or number of bytes of binary data.\n// The length of character data includes the trailing spaces. The length of binary data\n// includes binary zeros.\n//\n// Length is the Golang equivalent of length: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Length(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"length\", col))\n}\n\n// OctetLength - Calculates the byte length for the specified string column.\n//\n// OctetLength is the Golang equivalent of octet_length: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc OctetLength(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"octet_length\", col))\n}\n\n// BitLength - Calculates the bit length for the specified string column.\n//\n// BitLength is the Golang equivalent of bit_length: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitLength(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bit_length\", col))\n}\n\n// Translate - A function translate any character in the `srcCol` by a character in `matching`.\n// The characters in `replace` is corresponding to the characters in `matching`.\n// Translation will happen whenever any character in the string is matching with the character\n// in the `matching`.\n//\n// Translate is the Golang equivalent of translate: (srcCol: 'ColumnOrName', matching: str, replace: str) -> pyspark.sql.connect.column.Column\nfunc Translate(srcCol column.Column, matching string, replace string) column.Column {\n\tlit_matching := StringLit(matching)\n\tlit_replace := StringLit(replace)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"translate\", srcCol, lit_matching, lit_replace))\n}\n\n// ToBinary - Converts the input `col` to a binary value based on the supplied `format`.\n// The `format` can be a case-insensitive string literal of \"hex\", \"utf-8\", \"utf8\",\n// or \"base64\". By default, the binary format for conversion is \"hex\" if\n// `format` is omitted. The function returns NULL if at least one of the\n// input parameters is NULL.\n//\n// ToBinary is the Golang equivalent of to_binary: (col: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc ToBinary(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_binary\", col, format))\n}\n\n// ToChar - Convert `col` to a string based on the `format`.\n// Throws an exception if the conversion fails. The format can consist of the following\n// characters, case insensitive:\n// '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the\n// format string matches a sequence of digits in the input value, generating a result\n// string of the same length as the corresponding sequence in the format string.\n// The result string is left-padded with zeros if the 0/9 sequence comprises more digits\n// than the matching part of the decimal value, starts with 0, and is before the decimal\n// point. Otherwise, it is padded with spaces.\n// '.' or 'D': Specifies the position of the decimal point (optional, only allowed once).\n// ',' or 'G': Specifies the position of the grouping (thousands) separator (,).\n// There must be a 0 or 9 to the left and right of each grouping separator.\n// '$': Specifies the location of the $ currency sign. This character may only be specified once.\n// 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at\n// the beginning or end of the format string). Note that 'S' prints '+' for positive\n// values but 'MI' prints a space.\n// 'PR': Only allowed at the end of the format string; specifies that the result string\n// will be wrapped by angle brackets if the input value is negative.\n//\n// ToChar is the Golang equivalent of to_char: (col: 'ColumnOrName', format: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToChar(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_char\", col, format))\n}\n\n// ToVarchar - Convert `col` to a string based on the `format`.\n// Throws an exception if the conversion fails. The format can consist of the following\n// characters, case insensitive:\n// '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the\n// format string matches a sequence of digits in the input value, generating a result\n// string of the same length as the corresponding sequence in the format string.\n// The result string is left-padded with zeros if the 0/9 sequence comprises more digits\n// than the matching part of the decimal value, starts with 0, and is before the decimal\n// point. Otherwise, it is padded with spaces.\n// '.' or 'D': Specifies the position of the decimal point (optional, only allowed once).\n// ',' or 'G': Specifies the position of the grouping (thousands) separator (,).\n// There must be a 0 or 9 to the left and right of each grouping separator.\n// '$': Specifies the location of the $ currency sign. This character may only be specified once.\n// 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed once at\n// the beginning or end of the format string). Note that 'S' prints '+' for positive\n// values but 'MI' prints a space.\n// 'PR': Only allowed at the end of the format string; specifies that the result string\n// will be wrapped by angle brackets if the input value is negative.\n//\n// ToVarchar is the Golang equivalent of to_varchar: (col: 'ColumnOrName', format: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToVarchar(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_varchar\", col, format))\n}\n\n// ToNumber - Convert string 'col' to a number based on the string format 'format'.\n// Throws an exception if the conversion fails. The format can consist of the following\n// characters, case insensitive:\n// '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the\n// format string matches a sequence of digits in the input string. If the 0/9\n// sequence starts with 0 and is before the decimal point, it can only match a digit\n// sequence of the same size. Otherwise, if the sequence starts with 9 or is after\n// the decimal point, it can match a digit sequence that has the same or smaller size.\n// '.' or 'D': Specifies the position of the decimal point (optional, only allowed once).\n// ',' or 'G': Specifies the position of the grouping (thousands) separator (,).\n// There must be a 0 or 9 to the left and right of each grouping separator.\n// 'col' must match the grouping separator relevant for the size of the number.\n// '$': Specifies the location of the $ currency sign. This character may only be\n// specified once.\n// 'S' or 'MI': Specifies the position of a '-' or '+' sign (optional, only allowed\n// once at the beginning or end of the format string). Note that 'S' allows '-'\n// but 'MI' does not.\n// 'PR': Only allowed at the end of the format string; specifies that 'col' indicates a\n// negative number with wrapping angled brackets.\n//\n// ToNumber is the Golang equivalent of to_number: (col: 'ColumnOrName', format: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToNumber(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_number\", col, format))\n}\n\n// Replace - Replaces all occurrences of `search` with `replace`.\n//\n// Replace is the Golang equivalent of replace: (src: 'ColumnOrName', search: 'ColumnOrName', replace: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Replace(src column.Column, search column.Column, replace column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"replace\", src, search, replace))\n}\n\n// SplitPart - Splits `str` by delimiter and return requested part of the split (1-based).\n// If any input is null, returns null. if `partNum` is out of range of split parts,\n// returns empty string. If `partNum` is 0, throws an error. If `partNum` is negative,\n// the parts are counted backward from the end of the string.\n// If the `delimiter` is an empty string, the `str` is not split.\n//\n// SplitPart is the Golang equivalent of split_part: (src: 'ColumnOrName', delimiter: 'ColumnOrName', partNum: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc SplitPart(src column.Column, delimiter column.Column, partNum column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"split_part\", src, delimiter, partNum))\n}\n\n// Substr - Returns the substring of `str` that starts at `pos` and is of length `len`,\n// or the slice of byte array that starts at `pos` and is of length `len`.\n//\n// Substr is the Golang equivalent of substr: (str: 'ColumnOrName', pos: 'ColumnOrName', len: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Substr(str column.Column, pos column.Column, len column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"substr\", str, pos, len))\n}\n\n// ParseUrl - Extracts a part from a URL.\n//\n// ParseUrl is the Golang equivalent of parse_url: (url: 'ColumnOrName', partToExtract: 'ColumnOrName', key: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc ParseUrl(url column.Column, partToExtract column.Column, key column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"parse_url\", url, partToExtract, key))\n}\n\n// Printf - Formats the arguments in printf-style and returns the result as a string column.\n//\n// Printf is the Golang equivalent of printf: (format: 'ColumnOrName', *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Printf(format column.Column, cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, format)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"printf\", vals...))\n}\n\n// UrlDecode - Decodes a `str` in 'application/x-www-form-urlencoded' format\n// using a specific encoding scheme.\n//\n// UrlDecode is the Golang equivalent of url_decode: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UrlDecode(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"url_decode\", str))\n}\n\n// UrlEncode - Translates a string into 'application/x-www-form-urlencoded' format\n// using a specific encoding scheme.\n//\n// UrlEncode is the Golang equivalent of url_encode: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UrlEncode(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"url_encode\", str))\n}\n\n// Position - Returns the position of the first occurrence of `substr` in `str` after position `start`.\n// The given `start` and return value are 1-based.\n//\n// Position is the Golang equivalent of position: (substr: 'ColumnOrName', str: 'ColumnOrName', start: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Position(substr column.Column, str column.Column, start column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"position\", substr, str, start))\n}\n\n// Endswith - Returns a boolean. The value is True if str ends with suffix.\n// Returns NULL if either input expression is NULL. Otherwise, returns False.\n// Both str or suffix must be of STRING or BINARY type.\n//\n// Endswith is the Golang equivalent of endswith: (str: 'ColumnOrName', suffix: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Endswith(str column.Column, suffix column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"endswith\", str, suffix))\n}\n\n// Startswith - Returns a boolean. The value is True if str starts with prefix.\n// Returns NULL if either input expression is NULL. Otherwise, returns False.\n// Both str or prefix must be of STRING or BINARY type.\n//\n// Startswith is the Golang equivalent of startswith: (str: 'ColumnOrName', prefix: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Startswith(str column.Column, prefix column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"startswith\", str, prefix))\n}\n\n// Char - Returns the ASCII character having the binary equivalent to `col`. If col is larger than 256 the\n// result is equivalent to char(col % 256)\n//\n// Char is the Golang equivalent of char: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Char(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"char\", col))\n}\n\n// TryToBinary - This is a special version of `to_binary` that performs the same operation, but returns a NULL\n// value instead of raising an error if the conversion cannot be performed.\n//\n// TryToBinary is the Golang equivalent of try_to_binary: (col: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc TryToBinary(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_to_binary\", col, format))\n}\n\n// TryToNumber - Convert string 'col' to a number based on the string format `format`. Returns NULL if the\n// string 'col' does not match the expected format. The format follows the same semantics as the\n// to_number function.\n//\n// TryToNumber is the Golang equivalent of try_to_number: (col: 'ColumnOrName', format: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TryToNumber(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_to_number\", col, format))\n}\n\n// Btrim - Remove the leading and trailing `trim` characters from `str`.\n//\n// Btrim is the Golang equivalent of btrim: (str: 'ColumnOrName', trim: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Btrim(str column.Column, trim column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"btrim\", str, trim))\n}\n\n// CharLength - Returns the character length of string data or number of bytes of binary data.\n// The length of string data includes the trailing spaces.\n// The length of binary data includes binary zeros.\n//\n// CharLength is the Golang equivalent of char_length: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CharLength(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"char_length\", str))\n}\n\n// CharacterLength - Returns the character length of string data or number of bytes of binary data.\n// The length of string data includes the trailing spaces.\n// The length of binary data includes binary zeros.\n//\n// CharacterLength is the Golang equivalent of character_length: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CharacterLength(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"character_length\", str))\n}\n\n// Contains - Returns a boolean. The value is True if right is found inside left.\n// Returns NULL if either input expression is NULL. Otherwise, returns False.\n// Both left or right must be of STRING or BINARY type.\n//\n// Contains is the Golang equivalent of contains: (left: 'ColumnOrName', right: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Contains(left column.Column, right column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"contains\", left, right))\n}\n\n// Elt - Returns the `n`-th input, e.g., returns `input2` when `n` is 2.\n// The function returns NULL if the index exceeds the length of the array\n// and `spark.sql.ansi.enabled` is set to false. If `spark.sql.ansi.enabled` is set to true,\n// it throws ArrayIndexOutOfBoundsException for invalid indices.\n//\n// Elt is the Golang equivalent of elt: (*inputs: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Elt(inputs ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, inputs...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"elt\", vals...))\n}\n\n// FindInSet - Returns the index (1-based) of the given string (`str`) in the comma-delimited\n// list (`strArray`). Returns 0, if the string was not found or if the given string (`str`)\n// contains a comma.\n//\n// FindInSet is the Golang equivalent of find_in_set: (str: 'ColumnOrName', str_array: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc FindInSet(str column.Column, str_array column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"find_in_set\", str, str_array))\n}\n\n// TODO: like: (str: 'ColumnOrName', pattern: 'ColumnOrName', escapeChar: Optional[ForwardRef('Column')] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: ilike: (str: 'ColumnOrName', pattern: 'ColumnOrName', escapeChar: Optional[ForwardRef('Column')] = None) -> pyspark.sql.connect.column.Column\n\n// Lcase - Returns `str` with all characters changed to lowercase.\n//\n// Lcase is the Golang equivalent of lcase: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Lcase(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"lcase\", str))\n}\n\n// Ucase - Returns `str` with all characters changed to uppercase.\n//\n// Ucase is the Golang equivalent of ucase: (str: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ucase(str column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ucase\", str))\n}\n\n// Left - Returns the leftmost `len`(`len` can be string type) characters from the string `str`,\n// if `len` is less or equal than 0 the result is an empty string.\n//\n// Left is the Golang equivalent of left: (str: 'ColumnOrName', len: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Left(str column.Column, len column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"left\", str, len))\n}\n\n// Right - Returns the rightmost `len`(`len` can be string type) characters from the string `str`,\n// if `len` is less or equal than 0 the result is an empty string.\n//\n// Right is the Golang equivalent of right: (str: 'ColumnOrName', len: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Right(str column.Column, len column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"right\", str, len))\n}\n\n// Mask - Masks the given string value. This can be useful for creating copies of tables with sensitive\n// information removed.\n//\n// Mask is the Golang equivalent of mask: (col: 'ColumnOrName', upperChar: Optional[ForwardRef('ColumnOrName')] = None, lowerChar: Optional[ForwardRef('ColumnOrName')] = None, digitChar: Optional[ForwardRef('ColumnOrName')] = None, otherChar: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc Mask(col column.Column, upperChar column.Column, lowerChar column.Column,\n\tdigitChar column.Column, otherChar column.Column,\n) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"mask\", col, upperChar, lowerChar, digitChar, otherChar))\n}\n\n// Curdate - Returns the current date at the start of query evaluation as a :class:`DateType` column.\n// All calls of current_date within the same query return the same value.\n//\n// Curdate is the Golang equivalent of curdate: () -> pyspark.sql.connect.column.Column\nfunc Curdate() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"curdate\"))\n}\n\n// CurrentDate - Returns the current date at the start of query evaluation as a :class:`DateType` column.\n// All calls of current_date within the same query return the same value.\n//\n// CurrentDate is the Golang equivalent of current_date: () -> pyspark.sql.connect.column.Column\nfunc CurrentDate() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_date\"))\n}\n\n// CurrentTimestamp - Returns the current timestamp at the start of query evaluation as a :class:`TimestampType`\n// column. All calls of current_timestamp within the same query return the same value.\n//\n// CurrentTimestamp is the Golang equivalent of current_timestamp: () -> pyspark.sql.connect.column.Column\nfunc CurrentTimestamp() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_timestamp\"))\n}\n\n// Now - Returns the current timestamp at the start of query evaluation.\n//\n// Now is the Golang equivalent of now: () -> pyspark.sql.connect.column.Column\nfunc Now() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"now\"))\n}\n\n// CurrentTimezone - Returns the current session local timezone.\n//\n// CurrentTimezone is the Golang equivalent of current_timezone: () -> pyspark.sql.connect.column.Column\nfunc CurrentTimezone() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_timezone\"))\n}\n\n// Localtimestamp - Returns the current timestamp without time zone at the start of query evaluation\n// as a timestamp without time zone column. All calls of localtimestamp within the\n// same query return the same value.\n//\n// Localtimestamp is the Golang equivalent of localtimestamp: () -> pyspark.sql.connect.column.Column\nfunc Localtimestamp() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"localtimestamp\"))\n}\n\n// DateFormat - Converts a date/timestamp/string to a value of string in the format specified by the date\n// format given by the second argument.\n//\n// A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All\n// pattern letters of `datetime pattern`_. can be used.\n//\n// DateFormat is the Golang equivalent of date_format: (date: 'ColumnOrName', format: str) -> pyspark.sql.connect.column.Column\nfunc DateFormat(date column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_format\", date, lit_format))\n}\n\n// Year - Extract the year of a given date/timestamp as integer.\n//\n// Year is the Golang equivalent of year: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Year(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"year\", col))\n}\n\n// Quarter - Extract the quarter of a given date/timestamp as integer.\n//\n// Quarter is the Golang equivalent of quarter: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Quarter(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"quarter\", col))\n}\n\n// Month - Extract the month of a given date/timestamp as integer.\n//\n// Month is the Golang equivalent of month: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Month(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"month\", col))\n}\n\n// Dayofweek - Extract the day of the week of a given date/timestamp as integer.\n// Ranges from 1 for a Sunday through to 7 for a Saturday\n//\n// Dayofweek is the Golang equivalent of dayofweek: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Dayofweek(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"dayofweek\", col))\n}\n\n// Dayofmonth - Extract the day of the month of a given date/timestamp as integer.\n//\n// Dayofmonth is the Golang equivalent of dayofmonth: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Dayofmonth(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"dayofmonth\", col))\n}\n\n// Day - Extract the day of the month of a given date/timestamp as integer.\n//\n// Day is the Golang equivalent of day: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Day(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"day\", col))\n}\n\n// Dayofyear - Extract the day of the year of a given date/timestamp as integer.\n//\n// Dayofyear is the Golang equivalent of dayofyear: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Dayofyear(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"dayofyear\", col))\n}\n\n// Hour - Extract the hours of a given timestamp as integer.\n//\n// Hour is the Golang equivalent of hour: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Hour(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hour\", col))\n}\n\n// Minute - Extract the minutes of a given timestamp as integer.\n//\n// Minute is the Golang equivalent of minute: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Minute(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"minute\", col))\n}\n\n// Second - Extract the seconds of a given date as integer.\n//\n// Second is the Golang equivalent of second: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Second(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"second\", col))\n}\n\n// Weekofyear - Extract the week number of a given date as integer.\n// A week is considered to start on a Monday and week 1 is the first week with more than 3 days,\n// as defined by ISO 8601\n//\n// Weekofyear is the Golang equivalent of weekofyear: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Weekofyear(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"weekofyear\", col))\n}\n\n// Weekday - Returns the day of the week for date/timestamp (0 = Monday, 1 = Tuesday, ..., 6 = Sunday).\n//\n// Weekday is the Golang equivalent of weekday: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Weekday(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"weekday\", col))\n}\n\n// Extract - Extracts a part of the date/timestamp or interval source.\n//\n// Extract is the Golang equivalent of extract: (field: 'ColumnOrName', source: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Extract(field column.Column, source column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"extract\", field, source))\n}\n\n// DatePart is the Golang equivalent of date_part: (field: 'ColumnOrName', source: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DatePart(field column.Column, source column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_part\", field, source))\n}\n\n// Datepart is the Golang equivalent of datepart: (field: 'ColumnOrName', source: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Datepart(field column.Column, source column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"datepart\", field, source))\n}\n\n// MakeDate - Returns a column with a date built from the year, month and day columns.\n//\n// MakeDate is the Golang equivalent of make_date: (year: 'ColumnOrName', month: 'ColumnOrName', day: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MakeDate(year column.Column, month column.Column, day column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_date\", year, month, day))\n}\n\n// DateAdd - Returns the date that is `days` days after `start`. If `days` is a negative value\n// then these amount of days will be deducted from `start`.\n//\n// DateAdd is the Golang equivalent of date_add: (start: 'ColumnOrName', days: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc DateAdd(start column.Column, days column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_add\", start, days))\n}\n\n// Dateadd - Returns the date that is `days` days after `start`. If `days` is a negative value\n// then these amount of days will be deducted from `start`.\n//\n// Dateadd is the Golang equivalent of dateadd: (start: 'ColumnOrName', days: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc Dateadd(start column.Column, days column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"dateadd\", start, days))\n}\n\n// DateSub - Returns the date that is `days` days before `start`. If `days` is a negative value\n// then these amount of days will be added to `start`.\n//\n// DateSub is the Golang equivalent of date_sub: (start: 'ColumnOrName', days: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc DateSub(start column.Column, days column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_sub\", start, days))\n}\n\n// Datediff - Returns the number of days from `start` to `end`.\n//\n// Datediff is the Golang equivalent of datediff: (end: 'ColumnOrName', start: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Datediff(end column.Column, start column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"datediff\", end, start))\n}\n\n// DateDiff - Returns the number of days from `start` to `end`.\n//\n// DateDiff is the Golang equivalent of date_diff: (end: 'ColumnOrName', start: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DateDiff(end column.Column, start column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_diff\", end, start))\n}\n\n// DateFromUnixDate - Create date from the number of `days` since 1970-01-01.\n//\n// DateFromUnixDate is the Golang equivalent of date_from_unix_date: (days: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DateFromUnixDate(days column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_from_unix_date\", days))\n}\n\n// AddMonths - Returns the date that is `months` months after `start`. If `months` is a negative value\n// then these amount of months will be deducted from the `start`.\n//\n// AddMonths is the Golang equivalent of add_months: (start: 'ColumnOrName', months: Union[ForwardRef('ColumnOrName'), int]) -> pyspark.sql.connect.column.Column\nfunc AddMonths(start column.Column, months column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"add_months\", start, months))\n}\n\n// TODO: months_between: (date1: 'ColumnOrName', date2: 'ColumnOrName', roundOff: bool = True) -> pyspark.sql.connect.column.Column\n\n// ToDate - Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.DateType`\n// using the optionally specified format. Specify formats according to `datetime pattern`_.\n// By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format\n// is omitted. Equivalent to “col.cast(\"date\")“.\n//\n// ToDate is the Golang equivalent of to_date: (col: 'ColumnOrName', format: Optional[str] = None) -> pyspark.sql.connect.column.Column\nfunc ToDate(col column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_date\", col, lit_format))\n}\n\n// UnixDate - Returns the number of days since 1970-01-01.\n//\n// UnixDate is the Golang equivalent of unix_date: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UnixDate(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unix_date\", col))\n}\n\n// UnixMicros - Returns the number of microseconds since 1970-01-01 00:00:00 UTC.\n//\n// UnixMicros is the Golang equivalent of unix_micros: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UnixMicros(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unix_micros\", col))\n}\n\n// UnixMillis - Returns the number of milliseconds since 1970-01-01 00:00:00 UTC.\n// Truncates higher levels of precision.\n//\n// UnixMillis is the Golang equivalent of unix_millis: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UnixMillis(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unix_millis\", col))\n}\n\n// UnixSeconds - Returns the number of seconds since 1970-01-01 00:00:00 UTC.\n// Truncates higher levels of precision.\n//\n// UnixSeconds is the Golang equivalent of unix_seconds: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc UnixSeconds(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unix_seconds\", col))\n}\n\n// ToTimestamp - Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.TimestampType`\n// using the optionally specified format. Specify formats according to `datetime pattern`_.\n// By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format\n// is omitted. Equivalent to “col.cast(\"timestamp\")“.\n//\n// ToTimestamp is the Golang equivalent of to_timestamp: (col: 'ColumnOrName', format: Optional[str] = None) -> pyspark.sql.connect.column.Column\nfunc ToTimestamp(col column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_timestamp\", col, lit_format))\n}\n\n// TryToTimestamp - Parses the `col` with the `format` to a timestamp. The function always\n// returns null on an invalid input with/without ANSI SQL mode enabled. The result data type is\n// consistent with the value of configuration `spark.sql.timestampType`.\n//\n// TryToTimestamp is the Golang equivalent of try_to_timestamp: (col: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc TryToTimestamp(col column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_to_timestamp\", col, format))\n}\n\n// Xpath - Returns a string array of values within the nodes of xml that match the XPath expression.\n//\n// Xpath is the Golang equivalent of xpath: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Xpath(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath\", xml, path))\n}\n\n// XpathBoolean - Returns true if the XPath expression evaluates to true, or if a matching node is found.\n//\n// XpathBoolean is the Golang equivalent of xpath_boolean: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathBoolean(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_boolean\", xml, path))\n}\n\n// XpathDouble - Returns a double value, the value zero if no match is found,\n// or NaN if a match is found but the value is non-numeric.\n//\n// XpathDouble is the Golang equivalent of xpath_double: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathDouble(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_double\", xml, path))\n}\n\n// XpathNumber - Returns a double value, the value zero if no match is found,\n// or NaN if a match is found but the value is non-numeric.\n//\n// XpathNumber is the Golang equivalent of xpath_number: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathNumber(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_number\", xml, path))\n}\n\n// XpathFloat - Returns a float value, the value zero if no match is found,\n// or NaN if a match is found but the value is non-numeric.\n//\n// XpathFloat is the Golang equivalent of xpath_float: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathFloat(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_float\", xml, path))\n}\n\n// XpathInt - Returns an integer value, or the value zero if no match is found,\n// or a match is found but the value is non-numeric.\n//\n// XpathInt is the Golang equivalent of xpath_int: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathInt(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_int\", xml, path))\n}\n\n// XpathLong - Returns a long integer value, or the value zero if no match is found,\n// or a match is found but the value is non-numeric.\n//\n// XpathLong is the Golang equivalent of xpath_long: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathLong(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_long\", xml, path))\n}\n\n// XpathShort - Returns a short integer value, or the value zero if no match is found,\n// or a match is found but the value is non-numeric.\n//\n// XpathShort is the Golang equivalent of xpath_short: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathShort(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_short\", xml, path))\n}\n\n// XpathString - Returns the text contents of the first xml node that matches the XPath expression.\n//\n// XpathString is the Golang equivalent of xpath_string: (xml: 'ColumnOrName', path: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc XpathString(xml column.Column, path column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xpath_string\", xml, path))\n}\n\n// Trunc - Returns date truncated to the unit specified by the format.\n//\n// Trunc is the Golang equivalent of trunc: (date: 'ColumnOrName', format: str) -> pyspark.sql.connect.column.Column\nfunc Trunc(date column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"trunc\", date, lit_format))\n}\n\n// DateTrunc - Returns timestamp truncated to the unit specified by the format.\n//\n// DateTrunc is the Golang equivalent of date_trunc: (format: str, timestamp: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc DateTrunc(format string, timestamp column.Column) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"date_trunc\", lit_format, timestamp))\n}\n\n// NextDay - Returns the first date which is later than the value of the date column\n// based on second `week day` argument.\n//\n// NextDay is the Golang equivalent of next_day: (date: 'ColumnOrName', dayOfWeek: str) -> pyspark.sql.connect.column.Column\nfunc NextDay(date column.Column, dayOfWeek string) column.Column {\n\tlit_dayOfWeek := StringLit(dayOfWeek)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"next_day\", date, lit_dayOfWeek))\n}\n\n// LastDay - Returns the last day of the month which the given date belongs to.\n//\n// LastDay is the Golang equivalent of last_day: (date: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc LastDay(date column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"last_day\", date))\n}\n\n// FromUnixtime - Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string\n// representing the timestamp of that moment in the current system time zone in the given\n// format.\n//\n// FromUnixtime is the Golang equivalent of from_unixtime: (timestamp: 'ColumnOrName', format: str = 'yyyy-MM-dd HH:mm:ss') -> pyspark.sql.connect.column.Column\nfunc FromUnixtime(timestamp column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"from_unixtime\", timestamp, lit_format))\n}\n\n// UnixTimestamp - Convert time string with given pattern ('yyyy-MM-dd HH:mm:ss', by default)\n// to Unix time stamp (in seconds), using the default timezone and the default\n// locale, returns null if failed.\n//\n// if `timestamp` is None, then it returns current timestamp.\n//\n// UnixTimestamp is the Golang equivalent of unix_timestamp: (timestamp: Optional[ForwardRef('ColumnOrName')] = None, format: str = 'yyyy-MM-dd HH:mm:ss') -> pyspark.sql.connect.column.Column\nfunc UnixTimestamp(timestamp column.Column, format string) column.Column {\n\tlit_format := StringLit(format)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"unix_timestamp\", timestamp, lit_format))\n}\n\n// FromUtcTimestamp - This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function\n// takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and\n// renders that timestamp as a timestamp in the given time zone.\n//\n// However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not\n// timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to\n// the given timezone.\n//\n// This function may return confusing result if the input is a string with timezone, e.g.\n// '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp\n// according to the timezone in the string, and finally display the result by converting the\n// timestamp to string according to the session local timezone.\n//\n// FromUtcTimestamp is the Golang equivalent of from_utc_timestamp: (timestamp: 'ColumnOrName', tz: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc FromUtcTimestamp(timestamp column.Column, tz column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"from_utc_timestamp\", timestamp, tz))\n}\n\n// ToUtcTimestamp - This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function\n// takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given\n// timezone, and renders that timestamp as a timestamp in UTC.\n//\n// However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not\n// timezone-agnostic. So in Spark this function just shift the timestamp value from the given\n// timezone to UTC timezone.\n//\n// This function may return confusing result if the input is a string with timezone, e.g.\n// '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp\n// according to the timezone in the string, and finally display the result by converting the\n// timestamp to string according to the session local timezone.\n//\n// ToUtcTimestamp is the Golang equivalent of to_utc_timestamp: (timestamp: 'ColumnOrName', tz: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc ToUtcTimestamp(timestamp column.Column, tz column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_utc_timestamp\", timestamp, tz))\n}\n\n// TimestampSeconds - Converts the number of seconds from the Unix epoch (1970-01-01T00:00:00Z)\n// to a timestamp.\n//\n// TimestampSeconds is the Golang equivalent of timestamp_seconds: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TimestampSeconds(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"timestamp_seconds\", col))\n}\n\n// TimestampMillis - Creates timestamp from the number of milliseconds since UTC epoch.\n//\n// TimestampMillis is the Golang equivalent of timestamp_millis: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TimestampMillis(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"timestamp_millis\", col))\n}\n\n// TimestampMicros - Creates timestamp from the number of microseconds since UTC epoch.\n//\n// TimestampMicros is the Golang equivalent of timestamp_micros: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc TimestampMicros(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"timestamp_micros\", col))\n}\n\n// Window - Bucketize rows into one or more time windows given a timestamp specifying column. Window\n// starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window\n// [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in\n// the order of months are not supported.\n//\n// The time column must be of :class:`pyspark.sql.types.TimestampType`.\n//\n// Durations are provided as strings, e.g. '1 second', '1 day 12 hours', '2 minutes'. Valid\n// interval strings are 'week', 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'.\n// If the “slideDuration“ is not provided, the windows will be tumbling windows.\n//\n// The startTime is the offset with respect to 1970-01-01 00:00:00 UTC with which to start\n// window intervals. For example, in order to have hourly tumbling windows that start 15 minutes\n// past the hour, e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`.\n//\n// The output column will be a struct called 'window' by default with the nested columns 'start'\n// and 'end', where 'start' and 'end' will be of :class:`pyspark.sql.types.TimestampType`.\n//\n// Window is the Golang equivalent of window: (timeColumn: 'ColumnOrName', windowDuration: str, slideDuration: Optional[str] = None, startTime: Optional[str] = None) -> pyspark.sql.connect.column.Column\nfunc Window(timeColumn column.Column, windowDuration string, slideDuration string, startTime string) column.Column {\n\tlit_windowDuration := StringLit(windowDuration)\n\tlit_slideDuration := StringLit(slideDuration)\n\tlit_startTime := StringLit(startTime)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"window\", timeColumn,\n\t\tlit_windowDuration, lit_slideDuration, lit_startTime))\n}\n\n// WindowTime - Computes the event time from a window column. The column window values are produced\n// by window aggregating operators and are of type `STRUCT<start: TIMESTAMP, end: TIMESTAMP>`\n// where start is inclusive and end is exclusive. The event time of records produced by window\n// aggregating operators can be computed as “window_time(window)“ and are\n// “window.end - lit(1).alias(\"microsecond\")“ (as microsecond is the minimal supported event\n// time precision). The window column must be one produced by a window aggregating operator.\n//\n// WindowTime is the Golang equivalent of window_time: (windowColumn: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc WindowTime(windowColumn column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"window_time\", windowColumn))\n}\n\n// TODO: session_window: (timeColumn: 'ColumnOrName', gapDuration: Union[pyspark.sql.connect.column.Column, str]) -> pyspark.sql.connect.column.Column\n\n// ToUnixTimestamp - Returns the UNIX timestamp of the given time.\n//\n// ToUnixTimestamp is the Golang equivalent of to_unix_timestamp: (timestamp: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc ToUnixTimestamp(timestamp column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_unix_timestamp\", timestamp, format))\n}\n\n// ToTimestampLtz - Parses the `timestamp` with the `format` to a timestamp without time zone.\n// Returns null with invalid input.\n//\n// ToTimestampLtz is the Golang equivalent of to_timestamp_ltz: (timestamp: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc ToTimestampLtz(timestamp column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_timestamp_ltz\", timestamp, format))\n}\n\n// ToTimestampNtz - Parses the `timestamp` with the `format` to a timestamp without time zone.\n// Returns null with invalid input.\n//\n// ToTimestampNtz is the Golang equivalent of to_timestamp_ntz: (timestamp: 'ColumnOrName', format: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc ToTimestampNtz(timestamp column.Column, format column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"to_timestamp_ntz\", timestamp, format))\n}\n\n// TODO: bucket: (numBuckets: Union[pyspark.sql.connect.column.Column, int], col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\n\n// Years - Partition transform function: A transform for timestamps and dates\n// to partition data into years.\n//\n// Years is the Golang equivalent of years: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Years(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"years\", col))\n}\n\n// Months - Partition transform function: A transform for timestamps and dates\n// to partition data into months.\n//\n// Months is the Golang equivalent of months: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Months(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"months\", col))\n}\n\n// Days - Partition transform function: A transform for timestamps and dates\n// to partition data into days.\n//\n// Days is the Golang equivalent of days: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Days(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"days\", col))\n}\n\n// Hours - Partition transform function: A transform for timestamps\n// to partition data into hours.\n//\n// Hours is the Golang equivalent of hours: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Hours(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hours\", col))\n}\n\n// TODO: convert_timezone: (sourceTz: Optional[pyspark.sql.connect.column.Column], targetTz: pyspark.sql.connect.column.Column, sourceTs: 'ColumnOrName') -> pyspark.sql.connect.column.Column\n\n// MakeDtInterval - Make DayTimeIntervalType duration from days, hours, mins and secs.\n//\n// MakeDtInterval is the Golang equivalent of make_dt_interval: (days: Optional[ForwardRef('ColumnOrName')] = None, hours: Optional[ForwardRef('ColumnOrName')] = None, mins: Optional[ForwardRef('ColumnOrName')] = None, secs: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc MakeDtInterval(days column.Column, hours column.Column, mins column.Column, secs column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_dt_interval\", days, hours, mins, secs))\n}\n\n// MakeInterval - Make interval from years, months, weeks, days, hours, mins and secs.\n//\n// MakeInterval is the Golang equivalent of make_interval: (years: Optional[ForwardRef('ColumnOrName')] = None, months: Optional[ForwardRef('ColumnOrName')] = None, weeks: Optional[ForwardRef('ColumnOrName')] = None, days: Optional[ForwardRef('ColumnOrName')] = None, hours: Optional[ForwardRef('ColumnOrName')] = None, mins: Optional[ForwardRef('ColumnOrName')] = None, secs: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc MakeInterval(years column.Column, months column.Column, weeks column.Column,\n\tdays column.Column, hours column.Column, mins column.Column, secs column.Column,\n) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_interval\", years,\n\t\tmonths, weeks, days, hours, mins, secs))\n}\n\n// MakeTimestamp - Create timestamp from years, months, days, hours, mins, secs and timezone fields.\n// The result data type is consistent with the value of configuration `spark.sql.timestampType`.\n// If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL\n// on invalid inputs. Otherwise, it will throw an error instead.\n//\n// MakeTimestamp is the Golang equivalent of make_timestamp: (years: 'ColumnOrName', months: 'ColumnOrName', days: 'ColumnOrName', hours: 'ColumnOrName', mins: 'ColumnOrName', secs: 'ColumnOrName', timezone: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc MakeTimestamp(years column.Column, months column.Column, days column.Column,\n\thours column.Column, mins column.Column, secs column.Column, timezone column.Column,\n) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_timestamp\", years,\n\t\tmonths, days, hours, mins, secs, timezone))\n}\n\n// MakeTimestampLtz - Create the current timestamp with local time zone from years, months, days, hours, mins,\n// secs and timezone fields. If the configuration `spark.sql.ansi.enabled` is false,\n// the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.\n//\n// MakeTimestampLtz is the Golang equivalent of make_timestamp_ltz: (years: 'ColumnOrName', months: 'ColumnOrName', days: 'ColumnOrName', hours: 'ColumnOrName', mins: 'ColumnOrName', secs: 'ColumnOrName', timezone: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc MakeTimestampLtz(years column.Column, months column.Column, days column.Column,\n\thours column.Column, mins column.Column, secs column.Column, timezone column.Column,\n) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_timestamp_ltz\",\n\t\tyears, months, days, hours, mins, secs, timezone))\n}\n\n// MakeTimestampNtz - Create local date-time from years, months, days, hours, mins, secs fields.\n// If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL\n// on invalid inputs. Otherwise, it will throw an error instead.\n//\n// MakeTimestampNtz is the Golang equivalent of make_timestamp_ntz: (years: 'ColumnOrName', months: 'ColumnOrName', days: 'ColumnOrName', hours: 'ColumnOrName', mins: 'ColumnOrName', secs: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc MakeTimestampNtz(years column.Column, months column.Column, days column.Column,\n\thours column.Column, mins column.Column, secs column.Column,\n) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_timestamp_ntz\",\n\t\tyears, months, days, hours, mins, secs))\n}\n\n// MakeYmInterval - Make year-month interval from years, months.\n//\n// MakeYmInterval is the Golang equivalent of make_ym_interval: (years: Optional[ForwardRef('ColumnOrName')] = None, months: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc MakeYmInterval(years column.Column, months column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"make_ym_interval\", years, months))\n}\n\n// CurrentCatalog - Returns the current catalog.\n//\n// CurrentCatalog is the Golang equivalent of current_catalog: () -> pyspark.sql.connect.column.Column\nfunc CurrentCatalog() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_catalog\"))\n}\n\n// CurrentDatabase - Returns the current database.\n//\n// CurrentDatabase is the Golang equivalent of current_database: () -> pyspark.sql.connect.column.Column\nfunc CurrentDatabase() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_database\"))\n}\n\n// CurrentSchema - Returns the current database.\n//\n// CurrentSchema is the Golang equivalent of current_schema: () -> pyspark.sql.connect.column.Column\nfunc CurrentSchema() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_schema\"))\n}\n\n// CurrentUser - Returns the current database.\n//\n// CurrentUser is the Golang equivalent of current_user: () -> pyspark.sql.connect.column.Column\nfunc CurrentUser() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"current_user\"))\n}\n\n// User - Returns the current database.\n//\n// User is the Golang equivalent of user: () -> pyspark.sql.connect.column.Column\nfunc User() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"user\"))\n}\n\n// TODO: assert_true: (col: 'ColumnOrName', errMsg: Union[pyspark.sql.connect.column.Column, str, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: raise_error: (errMsg: Union[pyspark.sql.connect.column.Column, str]) -> pyspark.sql.connect.column.Column\n\n// Crc32 - Calculates the cyclic redundancy check value  (CRC32) of a binary column and\n// returns the value as a bigint.\n//\n// Crc32 is the Golang equivalent of crc32: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Crc32(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"crc32\", col))\n}\n\n// Hash - Calculates the hash code of given columns, and returns the result as an int column.\n//\n// Hash is the Golang equivalent of hash: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Hash(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hash\", vals...))\n}\n\n// Xxhash64 - Calculates the hash code of given columns using the 64-bit variant of the xxHash algorithm,\n// and returns the result as a long column. The hash computation uses an initial seed of 42.\n//\n// Xxhash64 is the Golang equivalent of xxhash64: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Xxhash64(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"xxhash64\", vals...))\n}\n\n// Md5 - Calculates the MD5 digest and returns the value as a 32 character hex string.\n//\n// Md5 is the Golang equivalent of md5: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Md5(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"md5\", col))\n}\n\n// Sha1 - Returns the hex string result of SHA-1.\n//\n// Sha1 is the Golang equivalent of sha1: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sha1(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sha1\", col))\n}\n\n// Sha2 - Returns the hex string result of SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384,\n// and SHA-512). The numBits indicates the desired bit length of the result, which must have a\n// value of 224, 256, 384, 512, or 0 (which is equivalent to 256).\n//\n// Sha2 is the Golang equivalent of sha2: (col: 'ColumnOrName', numBits: int) -> pyspark.sql.connect.column.Column\nfunc Sha2(col column.Column, numBits int64) column.Column {\n\tlit_numBits := Int64Lit(numBits)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sha2\", col, lit_numBits))\n}\n\n// TODO: hll_sketch_agg: (col: 'ColumnOrName', lgConfigK: Union[int, pyspark.sql.connect.column.Column, NoneType] = None) -> pyspark.sql.connect.column.Column\n\n// TODO: hll_union_agg: (col: 'ColumnOrName', allowDifferentLgConfigK: Optional[bool] = None) -> pyspark.sql.connect.column.Column\n\n// HllSketchEstimate - Returns the estimated number of unique values given the binary representation\n// of a Datasketches HllSketch.\n//\n// HllSketchEstimate is the Golang equivalent of hll_sketch_estimate: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc HllSketchEstimate(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"hll_sketch_estimate\", col))\n}\n\n// TODO: hll_union: (col1: 'ColumnOrName', col2: 'ColumnOrName', allowDifferentLgConfigK: Optional[bool] = None) -> pyspark.sql.connect.column.Column\n\n// Ifnull - Returns `col2` if `col1` is null, or `col1` otherwise.\n//\n// Ifnull is the Golang equivalent of ifnull: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Ifnull(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"ifnull\", col1, col2))\n}\n\n// Isnotnull - Returns true if `col` is not null, or false otherwise.\n//\n// Isnotnull is the Golang equivalent of isnotnull: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Isnotnull(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"isnotnull\", col))\n}\n\n// EqualNull - Returns same result as the EQUAL(=) operator for non-null operands,\n// but returns true if both are null, false if one of the them is null.\n//\n// EqualNull is the Golang equivalent of equal_null: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc EqualNull(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"equal_null\", col1, col2))\n}\n\n// Nullif - Returns null if `col1` equals to `col2`, or `col1` otherwise.\n//\n// Nullif is the Golang equivalent of nullif: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Nullif(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"nullif\", col1, col2))\n}\n\n// Nvl - Returns `col2` if `col1` is null, or `col1` otherwise.\n//\n// Nvl is the Golang equivalent of nvl: (col1: 'ColumnOrName', col2: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Nvl(col1 column.Column, col2 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"nvl\", col1, col2))\n}\n\n// Nvl2 - Returns `col2` if `col1` is not null, or `col3` otherwise.\n//\n// Nvl2 is the Golang equivalent of nvl2: (col1: 'ColumnOrName', col2: 'ColumnOrName', col3: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Nvl2(col1 column.Column, col2 column.Column, col3 column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"nvl2\", col1, col2, col3))\n}\n\n// AesEncrypt - Returns an encrypted value of `input` using AES in given `mode` with the specified `padding`.\n// Key lengths of 16, 24 and 32 bits are supported. Supported combinations of (`mode`,\n// `padding`) are ('ECB', 'PKCS'), ('GCM', 'NONE') and ('CBC', 'PKCS'). Optional initialization\n// vectors (IVs) are only supported for CBC and GCM modes. These must be 16 bytes for CBC and 12\n// bytes for GCM. If not provided, a random vector will be generated and prepended to the\n// output. Optional additional authenticated data (AAD) is only supported for GCM. If provided\n// for encryption, the identical AAD value must be provided for decryption. The default mode is\n// GCM.\n//\n// AesEncrypt is the Golang equivalent of aes_encrypt: (input: 'ColumnOrName', key: 'ColumnOrName', mode: Optional[ForwardRef('ColumnOrName')] = None, padding: Optional[ForwardRef('ColumnOrName')] = None, iv: Optional[ForwardRef('ColumnOrName')] = None, aad: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc AesEncrypt(input column.Column, key column.Column, mode column.Column, padding column.Column, iv column.Column, aad column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"aes_encrypt\", input, key, mode, padding, iv, aad))\n}\n\n// AesDecrypt - Returns a decrypted value of `input` using AES in `mode` with `padding`. Key lengths of 16,\n// 24 and 32 bits are supported. Supported combinations of (`mode`, `padding`) are ('ECB',\n// 'PKCS'), ('GCM', 'NONE') and ('CBC', 'PKCS'). Optional additional authenticated data (AAD) is\n// only supported for GCM. If provided for encryption, the identical AAD value must be provided\n// for decryption. The default mode is GCM.\n//\n// AesDecrypt is the Golang equivalent of aes_decrypt: (input: 'ColumnOrName', key: 'ColumnOrName', mode: Optional[ForwardRef('ColumnOrName')] = None, padding: Optional[ForwardRef('ColumnOrName')] = None, aad: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc AesDecrypt(input column.Column, key column.Column, mode column.Column, padding column.Column, aad column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"aes_decrypt\", input, key, mode, padding, aad))\n}\n\n// TryAesDecrypt - This is a special version of `aes_decrypt` that performs the same operation,\n// but returns a NULL value instead of raising an error if the decryption cannot be performed.\n// Returns a decrypted value of `input` using AES in `mode` with `padding`. Key lengths of 16,\n// 24 and 32 bits are supported. Supported combinations of (`mode`, `padding`) are ('ECB',\n// 'PKCS'), ('GCM', 'NONE') and ('CBC', 'PKCS'). Optional additional authenticated data (AAD) is\n// only supported for GCM. If provided for encryption, the identical AAD value must be provided\n// for decryption. The default mode is GCM.\n//\n// TryAesDecrypt is the Golang equivalent of try_aes_decrypt: (input: 'ColumnOrName', key: 'ColumnOrName', mode: Optional[ForwardRef('ColumnOrName')] = None, padding: Optional[ForwardRef('ColumnOrName')] = None, aad: Optional[ForwardRef('ColumnOrName')] = None) -> pyspark.sql.connect.column.Column\nfunc TryAesDecrypt(input column.Column, key column.Column, mode column.Column, padding column.Column, aad column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"try_aes_decrypt\", input, key, mode, padding, aad))\n}\n\n// Sha - Returns a sha1 hash value as a hex string of the `col`.\n//\n// Sha is the Golang equivalent of sha: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Sha(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"sha\", col))\n}\n\n// InputFileBlockLength - Returns the length of the block being read, or -1 if not available.\n//\n// InputFileBlockLength is the Golang equivalent of input_file_block_length: () -> pyspark.sql.connect.column.Column\nfunc InputFileBlockLength() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"input_file_block_length\"))\n}\n\n// InputFileBlockStart - Returns the start offset of the block being read, or -1 if not available.\n//\n// InputFileBlockStart is the Golang equivalent of input_file_block_start: () -> pyspark.sql.connect.column.Column\nfunc InputFileBlockStart() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"input_file_block_start\"))\n}\n\n// Reflect - Calls a method with reflection.\n//\n// Reflect is the Golang equivalent of reflect: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Reflect(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"reflect\", vals...))\n}\n\n// JavaMethod - Calls a method with reflection.\n//\n// JavaMethod is the Golang equivalent of java_method: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc JavaMethod(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"java_method\", vals...))\n}\n\n// Version - Returns the Spark version. The string contains 2 fields, the first being a release version\n// and the second being a git revision.\n//\n// Version is the Golang equivalent of version: () -> pyspark.sql.connect.column.Column\nfunc Version() column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"version\"))\n}\n\n// Typeof - Return DDL-formatted type string for the data type of the input.\n//\n// Typeof is the Golang equivalent of typeof: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Typeof(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"typeof\", col))\n}\n\n// Stack - Separates `col1`, ..., `colk` into `n` rows. Uses column names col0, col1, etc. by default\n// unless specified otherwise.\n//\n// Stack is the Golang equivalent of stack: (*cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc Stack(cols ...column.Column) column.Column {\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"stack\", vals...))\n}\n\n// BitmapBitPosition - Returns the bit position for the given input column.\n//\n// BitmapBitPosition is the Golang equivalent of bitmap_bit_position: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitmapBitPosition(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitmap_bit_position\", col))\n}\n\n// BitmapBucketNumber - Returns the bucket number for the given input column.\n//\n// BitmapBucketNumber is the Golang equivalent of bitmap_bucket_number: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitmapBucketNumber(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitmap_bucket_number\", col))\n}\n\n// BitmapConstructAgg - Returns a bitmap with the positions of the bits set from all the values from the input column.\n// The input column will most likely be bitmap_bit_position().\n//\n// BitmapConstructAgg is the Golang equivalent of bitmap_construct_agg: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitmapConstructAgg(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitmap_construct_agg\", col))\n}\n\n// BitmapCount - Returns the number of set bits in the input bitmap.\n//\n// BitmapCount is the Golang equivalent of bitmap_count: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitmapCount(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitmap_count\", col))\n}\n\n// BitmapOrAgg - Returns a bitmap that is the bitwise OR of all of the bitmaps from the input column.\n// The input column should be bitmaps created from bitmap_construct_agg().\n//\n// BitmapOrAgg is the Golang equivalent of bitmap_or_agg: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc BitmapOrAgg(col column.Column) column.Column {\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"bitmap_or_agg\", col))\n}\n\n// Ignore UDF: call_udf: (udfName: str, *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\n\n// Ignore UDT: unwrap_udt: (col: 'ColumnOrName') -> pyspark.sql.connect.column.Column\n\n// TODO: udf: (f: Union[Callable[..., Any], ForwardRef('DataTypeOrString'), NoneType] = None, returnType: 'DataTypeOrString' = StringType(), useArrow: Optional[bool] = None) -> Union[ForwardRef('UserDefinedFunctionLike'), Callable[[Callable[..., Any]], ForwardRef('UserDefinedFunctionLike')]]\n\n// Ignore UDT: udtf: (cls: Optional[Type] = None, *, returnType: Union[pyspark.sql.types.StructType, str], useArrow: Optional[bool] = None) -> Union[ForwardRef('UserDefinedTableFunction'), Callable[[Type], ForwardRef('UserDefinedTableFunction')]]\n\n// CallFunction - Call a SQL function.\n//\n// CallFunction is the Golang equivalent of call_function: (funcName: str, *cols: 'ColumnOrName') -> pyspark.sql.connect.column.Column\nfunc CallFunction(funcName string, cols ...column.Column) column.Column {\n\tlit_funcName := StringLit(funcName)\n\tvals := make([]column.Column, 0)\n\tvals = append(vals, lit_funcName)\n\tvals = append(vals, cols...)\n\treturn column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"call_function\", vals...))\n}\n"
  },
  {
    "path": "spark/sql/group.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"github.com/apache/spark-connect-go/spark/sql/column\"\n\t\"github.com/apache/spark-connect-go/spark/sql/functions\"\n)\n\ntype GroupedData struct {\n\tdf           *dataFrameImpl\n\tgroupType    string\n\tgroupingCols []column.Convertible\n\tpivotValues  []types.LiteralType\n\tpivotCol     column.Convertible\n}\n\n// Agg compute aggregates and returns the result as a DataFrame. The aggegrate expressions\n// are passed as column.Column arguments.\nfunc (gd *GroupedData) Agg(ctx context.Context, exprs ...column.Convertible) (DataFrame, error) {\n\tif len(exprs) == 0 {\n\t\treturn nil, sparkerrors.WithString(sparkerrors.InvalidInputError, \"exprs should not be empty\")\n\t}\n\n\tagg := &proto.Aggregate{\n\t\tInput: gd.df.relation,\n\t}\n\n\t// Add all grouping and aggregate expressions.\n\tagg.GroupingExpressions = make([]*proto.Expression, len(gd.groupingCols))\n\tfor i, col := range gd.groupingCols {\n\t\texp, err := col.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tagg.GroupingExpressions[i] = exp\n\t}\n\n\tagg.AggregateExpressions = make([]*proto.Expression, len(exprs))\n\tfor i, expr := range exprs {\n\t\texp, err := expr.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tagg.AggregateExpressions[i] = exp\n\t}\n\n\t// Apply the groupType\n\tswitch gd.groupType {\n\tcase \"pivot\":\n\t\tagg.GroupType = proto.Aggregate_GROUP_TYPE_PIVOT\n\t\t// Apply all pivot behavior and convert columns into literals.\n\t\tif len(gd.pivotValues) == 0 {\n\t\t\treturn nil, sparkerrors.WithString(sparkerrors.InvalidInputError, \"pivotValues should not be empty\")\n\t\t}\n\t\tprotoCol, err := gd.pivotCol.ToProto(ctx)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\n\t\tagg.Pivot = &proto.Aggregate_Pivot{\n\t\t\tValues: make([]*proto.Expression_Literal, len(gd.pivotValues)),\n\t\t\tCol:    protoCol,\n\t\t}\n\t\tfor i, v := range gd.pivotValues {\n\t\t\texp, err := column.NewLiteral(v).ToProto(ctx)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, err\n\t\t\t}\n\t\t\tagg.Pivot.Values[i] = exp.GetLiteral()\n\t\t}\n\tcase \"groupby\":\n\t\tagg.GroupType = proto.Aggregate_GROUP_TYPE_GROUPBY\n\tcase \"rollup\":\n\t\tagg.GroupType = proto.Aggregate_GROUP_TYPE_ROLLUP\n\tcase \"cube\":\n\t\tagg.GroupType = proto.Aggregate_GROUP_TYPE_CUBE\n\t}\n\n\trel := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Aggregate{\n\t\t\tAggregate: agg,\n\t\t},\n\t}\n\treturn NewDataFrame(gd.df.session, rel), nil\n}\n\nfunc (gd *GroupedData) numericAgg(ctx context.Context, name string, cols ...string) (DataFrame, error) {\n\tschema, err := gd.df.Schema(ctx)\n\tif err != nil {\n\t\treturn nil, err\n\t}\n\n\t// Find all numeric cols in the schema:\n\tnumericCols := make([]string, 0)\n\tfor _, field := range schema.Fields {\n\t\tif field.DataType.IsNumeric() {\n\t\t\tnumericCols = append(numericCols, field.Name)\n\t\t}\n\t}\n\n\taggCols := cols\n\tif len(cols) > 0 {\n\t\tinvalidCols := make([]string, 0)\n\t\tfor _, col := range cols {\n\t\t\tfound := false\n\t\t\tfor _, nc := range numericCols {\n\t\t\t\tif col == nc {\n\t\t\t\t\tfound = true\n\t\t\t\t}\n\t\t\t}\n\t\t\tif !found {\n\t\t\t\tinvalidCols = append(invalidCols, col)\n\t\t\t}\n\t\t}\n\t\tif len(invalidCols) > 0 {\n\t\t\treturn nil, sparkerrors.WithStringf(sparkerrors.InvalidInputError,\n\t\t\t\t\"columns %v are not numeric\", invalidCols)\n\t\t}\n\t} else {\n\t\taggCols = numericCols\n\t}\n\n\tfinalColumns := make([]column.Convertible, len(aggCols))\n\tfor i, col := range aggCols {\n\t\tfinalColumns[i] = column.NewColumn(column.NewUnresolvedFunctionWithColumns(name, functions.Col(col)))\n\t}\n\treturn gd.Agg(ctx, finalColumns...)\n}\n\n// Min Computes the min value for each numeric column for each group.\nfunc (gd *GroupedData) Min(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn gd.numericAgg(ctx, \"min\", cols...)\n}\n\n// Max Computes the max value for each numeric column for each group.\nfunc (gd *GroupedData) Max(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn gd.numericAgg(ctx, \"max\", cols...)\n}\n\n// Avg Computes the avg value for each numeric column for each group.\nfunc (gd *GroupedData) Avg(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn gd.numericAgg(ctx, \"avg\", cols...)\n}\n\n// Sum Computes the sum value for each numeric column for each group.\nfunc (gd *GroupedData) Sum(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn gd.numericAgg(ctx, \"sum\", cols...)\n}\n\n// Count Computes the count value for each group.\nfunc (gd *GroupedData) Count(ctx context.Context) (DataFrame, error) {\n\treturn gd.Agg(ctx, functions.Count(functions.Lit(types.Int64(1))).Alias(\"count\"))\n}\n\n// Mean Computes the average value for each numeric column for each group.\nfunc (gd *GroupedData) Mean(ctx context.Context, cols ...string) (DataFrame, error) {\n\treturn gd.Avg(ctx, cols...)\n}\n\nfunc (gd *GroupedData) Pivot(ctx context.Context, pivotCol string, pivotValues []types.LiteralType) (*GroupedData, error) {\n\tif gd.groupType != \"groupby\" {\n\t\tif gd.groupType == \"pivot\" {\n\t\t\treturn nil, sparkerrors.WithString(sparkerrors.InvalidInputError, \"pivot cannot be applied on pivot\")\n\t\t}\n\t\treturn nil, sparkerrors.WithString(sparkerrors.InvalidInputError, \"pivot can only be applied on groupby\")\n\t}\n\treturn &GroupedData{\n\t\tdf:           gd.df,\n\t\tgroupType:    \"pivot\",\n\t\tgroupingCols: gd.groupingCols,\n\t\tpivotValues:  pivotValues,\n\t\tpivotCol:     column.NewColumnReferenceWithPlanId(pivotCol, gd.df.PlanId()),\n\t}, nil\n}\n"
  },
  {
    "path": "spark/sql/group_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/client\"\n\t\"github.com/apache/spark-connect-go/spark/client/testutils\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nvar sampleDataFrame = &dataFrameImpl{session: nil, relation: &proto.Relation{\n\tRelType: &proto.Relation_Range{\n\t\tRange: &proto.Range{\n\t\t\tEnd:  10,\n\t\t\tStep: 1,\n\t\t},\n\t},\n}}\n\nfunc TestGroupedData_Agg(t *testing.T) {\n\tctx := context.Background()\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(nil, mocks.AnalyzePlanResponse, nil, nil), nil, mocks.MockSessionId)\n\tsession := sparkSessionImpl{sessionId: mocks.MockSessionId, client: c}\n\tsampleDataFrame.session = &session\n\n\tgd := GroupedData{\n\t\tgroupType: \"groupby\",\n\t\tdf:        sampleDataFrame,\n\t}\n\n\t// Should not be able to group by a non-existing column\n\t_, err := gd.Min(ctx, \"nonExistingColumn\")\n\tassert.Error(t, err)\n\n\t// Group by an existing column should work\n\tdf, err := gd.Min(ctx, \"col0\")\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Equal(t, \"min\", df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions()[0].GetUnresolvedFunction().FunctionName)\n\n\t// Group by an existing column should work\n\tdf, err = gd.Max(ctx, \"col0\")\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Equal(t, \"max\", df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions()[0].GetUnresolvedFunction().FunctionName)\n\n\tdf, err = gd.Sum(ctx, \"col0\")\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Equal(t, \"sum\", df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions()[0].GetUnresolvedFunction().FunctionName)\n\n\tdf, err = gd.Avg(ctx, \"col0\")\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Equal(t, \"avg\", df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions()[0].GetUnresolvedFunction().FunctionName)\n\n\t// Group by no column should pick all numeric columns\n\tdf, err = gd.Min(ctx)\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Len(t, df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions(), 1)\n}\n\nfunc TestGroupedData_Count(t *testing.T) {\n\tctx := context.Background()\n\tc := client.NewSparkExecutorFromClient(\n\t\ttestutils.NewConnectServiceClientMock(nil, mocks.AnalyzePlanResponse, nil, nil), nil, mocks.MockSessionId)\n\tsession := sparkSessionImpl{sessionId: mocks.MockSessionId, client: c}\n\tsampleDataFrame.session = &session\n\n\tgd := GroupedData{\n\t\tgroupType: \"groupby\",\n\t\tdf:        sampleDataFrame,\n\t}\n\n\tdf, err := gd.Count(ctx)\n\tassert.NoError(t, err)\n\tassert.IsType(t, df.(*dataFrameImpl).relation.RelType, &proto.Relation_Aggregate{})\n\tassert.Equal(t, []string{\"count\"}, df.(*dataFrameImpl).relation.GetAggregate().GetAggregateExpressions()[0].GetAlias().Name)\n}\n"
  },
  {
    "path": "spark/sql/mocks_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n"
  },
  {
    "path": "spark/sql/plan.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"sync/atomic\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n)\n\nvar atomicInt64 atomic.Int64\n\nfunc newPlanId() *int64 {\n\tv := atomicInt64.Add(1)\n\treturn &v\n}\n\nfunc resetPlanIdForTesting() {\n\tatomicInt64.Swap(0)\n}\n\nfunc newReadTableRelation(table string) *proto.Relation {\n\treturn &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_Read{\n\t\t\tRead: &proto.Read{\n\t\t\t\tReadType: &proto.Read_NamedTable_{\n\t\t\t\t\tNamedTable: &proto.Read_NamedTable{\n\t\t\t\t\t\tUnparsedIdentifier: table,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n}\n\nfunc newReadWithFormatAndPath(path, format string) *proto.Relation {\n\treturn &proto.Relation{\n\t\tRelType: &proto.Relation_Read{\n\t\t\tRead: &proto.Read{\n\t\t\t\tReadType: &proto.Read_DataSource_{\n\t\t\t\t\tDataSource: &proto.Read_DataSource{\n\t\t\t\t\t\tFormat: &format,\n\t\t\t\t\t\tPaths:  []string{path},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n}\n\nfunc newReadWithFormatAndPathAndOptions(path, format string, options map[string]string) *proto.Relation {\n\treturn &proto.Relation{\n\t\tRelType: &proto.Relation_Read{\n\t\t\tRead: &proto.Read{\n\t\t\t\tReadType: &proto.Read_DataSource_{\n\t\t\t\t\tDataSource: &proto.Read_DataSource{\n\t\t\t\t\t\tFormat:  &format,\n\t\t\t\t\t\tPaths:   []string{path},\n\t\t\t\t\t\tOptions: options,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n}\n"
  },
  {
    "path": "spark/sql/plan_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"testing\"\n\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestNewPlanIdGivesNewIDs(t *testing.T) {\n\tid1 := newPlanId()\n\tid2 := newPlanId()\n\tassert.NotEqual(t, id1, id2)\n}\n"
  },
  {
    "path": "spark/sql/sparksession.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"bytes\"\n\t\"context\"\n\t\"fmt\"\n\t\"time\"\n\n\t\"github.com/apache/arrow-go/v18/arrow/memory\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/arrow-go/v18/arrow/ipc\"\n\t\"github.com/apache/spark-connect-go/spark/client/base\"\n\n\t\"github.com/apache/spark-connect-go/spark/client/options\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/client\"\n\t\"github.com/apache/spark-connect-go/spark/client/channel\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n\t\"github.com/google/uuid\"\n\t\"google.golang.org/grpc/metadata\"\n)\n\ntype SparkSession interface {\n\tRead() DataFrameReader\n\tSql(ctx context.Context, query string) (DataFrame, error)\n\tStop() error\n\tTable(name string) (DataFrame, error)\n\tCreateDataFrameFromArrow(ctx context.Context, data arrow.Table) (DataFrame, error)\n\tCreateDataFrame(ctx context.Context, data [][]any, schema *types.StructType) (DataFrame, error)\n\tConfig() client.RuntimeConfig\n}\n\n// NewSessionBuilder creates a new session builder for starting a new spark session\nfunc NewSessionBuilder() *SparkSessionBuilder {\n\treturn &SparkSessionBuilder{}\n}\n\ntype SparkSessionBuilder struct {\n\tconnectionString string\n\tchannelBuilder   channel.Builder\n}\n\n// Remote sets the connection string for remote connection\nfunc (s *SparkSessionBuilder) Remote(connectionString string) *SparkSessionBuilder {\n\ts.connectionString = connectionString\n\treturn s\n}\n\nfunc (s *SparkSessionBuilder) WithChannelBuilder(cb channel.Builder) *SparkSessionBuilder {\n\ts.channelBuilder = cb\n\treturn s\n}\n\nfunc (s *SparkSessionBuilder) Build(ctx context.Context) (SparkSession, error) {\n\tif s.channelBuilder == nil {\n\t\tcb, err := channel.NewBuilder(s.connectionString)\n\t\tif err != nil {\n\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\t\"failed to connect to remote %s: %w\", s.connectionString, err), sparkerrors.ConnectionError)\n\t\t}\n\t\ts.channelBuilder = cb\n\t}\n\tconn, err := s.channelBuilder.Build(ctx)\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to connect to remote %s: %w\",\n\t\t\ts.connectionString, err), sparkerrors.ConnectionError)\n\t}\n\n\t// Add metadata to the request.\n\tmeta := metadata.MD{}\n\tfor k, v := range s.channelBuilder.Headers() {\n\t\tmeta[k] = append(meta[k], v)\n\t}\n\n\tsessionId := uuid.NewString()\n\n\t// Update the options according to the configuration.\n\topts := options.NewSparkClientOptions(options.DefaultSparkClientOptions.ReattachExecution)\n\topts.UserAgent = s.channelBuilder.UserAgent()\n\topts.UserId = s.channelBuilder.User()\n\n\treturn &sparkSessionImpl{\n\t\tsessionId: sessionId,\n\t\tclient:    client.NewSparkExecutor(conn, meta, sessionId, opts),\n\t}, nil\n}\n\ntype sparkSessionImpl struct {\n\tsessionId string\n\tclient    base.SparkConnectClient\n}\n\nfunc (s *sparkSessionImpl) Config() client.RuntimeConfig {\n\treturn client.NewRuntimeConfig(&s.client)\n}\n\nfunc (s *sparkSessionImpl) Read() DataFrameReader {\n\treturn NewDataframeReader(s)\n}\n\n// Sql executes a sql query and returns the result as a DataFrame\nfunc (s *sparkSessionImpl) Sql(ctx context.Context, query string) (DataFrame, error) {\n\t// Due to the nature of Spark, we have to first submit the SQL query immediately as a command\n\t// to make sure that all side effects have been executed properly. If no side effects are present,\n\t// then simply prepare this as a SQL relation.\n\n\tplan := &proto.Plan{\n\t\tOpType: &proto.Plan_Command{\n\t\t\tCommand: &proto.Command{\n\t\t\t\tCommandType: &proto.Command_SqlCommand{\n\t\t\t\t\tSqlCommand: &proto.SqlCommand{\n\t\t\t\t\t\tSql: query,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\t// We need an execute command here.\n\t_, _, properties, err := s.client.ExecuteCommand(ctx, plan)\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to execute sql: %s: %w\", query, err), sparkerrors.ExecutionError)\n\t}\n\n\tval, ok := properties[\"sql_command_result\"]\n\tif !ok {\n\t\tplan := &proto.Relation{\n\t\t\tCommon: &proto.RelationCommon{\n\t\t\t\tPlanId: newPlanId(),\n\t\t\t},\n\t\t\tRelType: &proto.Relation_Sql{\n\t\t\t\tSql: &proto.SQL{\n\t\t\t\t\tQuery: query,\n\t\t\t\t},\n\t\t\t},\n\t\t}\n\t\treturn NewDataFrame(s, plan), nil\n\t} else {\n\t\trel := val.(*proto.Relation)\n\t\trel.Common = &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t}\n\t\treturn NewDataFrame(s, rel), nil\n\t}\n}\n\nfunc (s *sparkSessionImpl) Stop() error {\n\treturn nil\n}\n\nfunc (s *sparkSessionImpl) Table(name string) (DataFrame, error) {\n\treturn s.Read().Table(name)\n}\n\nfunc (s *sparkSessionImpl) CreateDataFrameFromArrow(ctx context.Context, data arrow.Table) (DataFrame, error) {\n\t// Generate the schema.\n\t// schema := types.ArrowSchemaToProto(data.Schema())\n\t// schemaString := \"\"\n\t// TODO (PySpark does a lot of casting here to convert the schema that does not exist yet.\n\n\t// Convert the Arrow Table into a byte array of arrow IPC messages.\n\tbuf := new(bytes.Buffer)\n\tw := ipc.NewWriter(buf, ipc.WithSchema(data.Schema()))\n\tdefer w.Close()\n\n\t// Create a RecordReader from the table\n\trr := array.NewTableReader(data, int64(data.NumRows()))\n\tdefer rr.Release()\n\n\t// Read the records from the table and write them to the buffer\n\tfor rr.Next() {\n\t\trecord := rr.Record()\n\t\tif err := w.Write(record); err != nil {\n\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to write record: %w\", err), sparkerrors.WriteError)\n\t\t}\n\t}\n\n\t// Create a local relation object\n\tplan := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_LocalRelation{\n\t\t\tLocalRelation: &proto.LocalRelation{\n\t\t\t\t// Schema: &schemaString,\n\t\t\t\tData: buf.Bytes(),\n\t\t\t},\n\t\t},\n\t}\n\n\t// Capture the column names from the schema:\n\tcolumnNames := make([]string, data.NumCols())\n\tfor i, field := range data.Schema().Fields() {\n\t\tcolumnNames[i] = field.Name\n\t}\n\n\tdfPlan := &proto.Relation{\n\t\tCommon: &proto.RelationCommon{\n\t\t\tPlanId: newPlanId(),\n\t\t},\n\t\tRelType: &proto.Relation_ToDf{\n\t\t\tToDf: &proto.ToDF{\n\t\t\t\tInput:       plan,\n\t\t\t\tColumnNames: columnNames,\n\t\t\t},\n\t\t},\n\t}\n\treturn NewDataFrame(s, dfPlan), nil\n}\n\nfunc (s *sparkSessionImpl) CreateDataFrame(ctx context.Context, data [][]any, schema *types.StructType) (DataFrame, error) {\n\tpool := memory.NewGoAllocator()\n\t// Convert the data into an Arrow Table\n\tarrowSchema := arrow.NewSchema(schema.ToArrowType().(*arrow.StructType).Fields(), nil)\n\trb := array.NewRecordBuilder(pool, arrowSchema)\n\tdefer rb.Release()\n\t// Iterate over all fields and add the values:\n\tfor _, row := range data {\n\t\tfor i, field := range schema.Fields {\n\t\t\tif row[i] == nil {\n\t\t\t\trb.Field(i).AppendNull()\n\t\t\t\tcontinue\n\t\t\t}\n\t\t\tswitch field.DataType {\n\t\t\tcase types.BOOLEAN:\n\t\t\t\trb.Field(i).(*array.BooleanBuilder).Append(row[i].(bool))\n\t\t\tcase types.BYTE:\n\t\t\t\trb.Field(i).(*array.Int8Builder).Append(int8(row[i].(int)))\n\t\t\tcase types.SHORT:\n\t\t\t\trb.Field(i).(*array.Int16Builder).Append(int16(row[i].(int)))\n\t\t\tcase types.INTEGER:\n\t\t\t\trb.Field(i).(*array.Int32Builder).Append(int32(row[i].(int)))\n\t\t\tcase types.LONG:\n\t\t\t\trb.Field(i).(*array.Int64Builder).Append(int64(row[i].(int)))\n\t\t\tcase types.FLOAT:\n\t\t\t\trb.Field(i).(*array.Float32Builder).Append(float32(row[i].(float32)))\n\t\t\tcase types.DOUBLE:\n\t\t\t\trb.Field(i).(*array.Float64Builder).Append(row[i].(float64))\n\t\t\tcase types.STRING:\n\t\t\t\trb.Field(i).(*array.StringBuilder).Append(row[i].(string))\n\t\t\tcase types.DATE:\n\t\t\t\trb.Field(i).(*array.Date32Builder).Append(\n\t\t\t\t\tarrow.Date32FromTime(row[i].(time.Time)))\n\t\t\tcase types.TIMESTAMP:\n\t\t\t\tts, err := arrow.TimestampFromTime(row[i].(time.Time), arrow.Millisecond)\n\t\t\t\tif err != nil {\n\t\t\t\t\treturn nil, err\n\t\t\t\t}\n\t\t\t\trb.Field(i).(*array.TimestampBuilder).Append(ts)\n\t\t\tdefault:\n\t\t\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\n\t\t\t\t\t\"unsupported data type: %s\", field.DataType), sparkerrors.NotImplementedError)\n\t\t\t}\n\t\t}\n\t}\n\trec := rb.NewRecord()\n\tdefer rec.Release()\n\ttbl := array.NewTableFromRecords(arrowSchema, []arrow.Record{rec})\n\tdefer tbl.Release()\n\treturn s.CreateDataFrameFromArrow(ctx, tbl)\n}\n"
  },
  {
    "path": "spark/sql/sparksession_integration_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"testing\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/memory\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestSparkSession_CreateDataFrame_StructTypeToArrowConversion(t *testing.T) {\n\t// This test validates that the fix for StructType.ToArrowType() works correctly\n\t// The change from .Fields() to .(*arrow.StructType).Fields() should be validated\n\n\t// Create a test schema\n\tschema := types.StructOf(\n\t\ttypes.NewStructField(\"id\", types.INTEGER),\n\t\ttypes.NewStructField(\"name\", types.STRING),\n\t\ttypes.NewStructField(\"scores\", types.ArrayType{\n\t\t\tElementType:  types.DOUBLE,\n\t\t\tContainsNull: true,\n\t\t}),\n\t)\n\n\t// Test that ToArrowType returns the correct interface type\n\tarrowType := schema.ToArrowType()\n\tassert.NotNil(t, arrowType)\n\n\t// Verify we can cast it to *arrow.StructType and access Fields()\n\tstructType, ok := arrowType.(*arrow.StructType)\n\tassert.True(t, ok)\n\n\tfields := structType.Fields()\n\tassert.Len(t, fields, 3)\n\tassert.Equal(t, \"id\", fields[0].Name)\n\tassert.Equal(t, \"name\", fields[1].Name)\n\tassert.Equal(t, \"scores\", fields[2].Name)\n\n\t// Test sample data that would work with CreateDataFrame\n\tdata := [][]any{\n\t\t{1, \"Alice\", []float64{95.5, 87.2, 92.1}},\n\t\t{2, \"Bob\", []float64{88.0, 91.5, 89.3}},\n\t}\n\n\t// Verify that the data structure is compatible with the schema\n\tassert.Len(t, data, 2)\n\tfor _, row := range data {\n\t\tassert.Len(t, row, 3)\n\t\tassert.IsType(t, 1, row[0])           // integer id\n\t\tassert.IsType(t, \"\", row[1])          // string name\n\t\tassert.IsType(t, []float64{}, row[2]) // array of doubles\n\t}\n\n\t// Test that we can create an Arrow schema using the converted type\n\tpool := memory.NewGoAllocator()\n\t_ = pool // Use the pool variable to avoid unused error\n\n\t// This would previously fail due to the type assertion issue\n\t// Now it should work because ToArrowType() returns arrow.DataType interface\n\tarrowSchema := arrow.NewSchema(structType.Fields(), nil)\n\tassert.NotNil(t, arrowSchema)\n\tassert.Equal(t, 3, len(arrowSchema.Fields()))\n}\n"
  },
  {
    "path": "spark/sql/sparksession_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage sql\n\nimport (\n\t\"bytes\"\n\t\"context\"\n\t\"io\"\n\t\"testing\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/arrow-go/v18/arrow/ipc\"\n\t\"github.com/apache/arrow-go/v18/arrow/memory\"\n\t\"github.com/stretchr/testify/assert\"\n\t\"github.com/stretchr/testify/require\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/client\"\n\t\"github.com/apache/spark-connect-go/spark/client/testutils\"\n\t\"github.com/apache/spark-connect-go/spark/mocks\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\nfunc TestSparkSessionTable(t *testing.T) {\n\tresetPlanIdForTesting()\n\tplan := newReadTableRelation(\"table\")\n\tresetPlanIdForTesting()\n\ts := testutils.NewConnectServiceClientMock(nil, nil, nil, t)\n\tc := client.NewSparkExecutorFromClient(s, nil, \"\")\n\tsession := &sparkSessionImpl{client: c}\n\tdf, err := session.Table(\"table\")\n\tdf_plan := df.(*dataFrameImpl).relation\n\tassert.Equal(t, plan, df_plan)\n\tassert.NoError(t, err)\n}\n\nfunc TestSQLCallsExecutePlanWithSQLOnClient(t *testing.T) {\n\tctx := context.Background()\n\n\tquery := \"select * from bla\"\n\t// Create the responses:\n\tresponses := []*mocks.MockResponse{\n\t\t{\n\t\t\tResp: &proto.ExecutePlanResponse{\n\t\t\t\tResponseType: &proto.ExecutePlanResponse_SqlCommandResult_{\n\t\t\t\t\tSqlCommandResult: &proto.ExecutePlanResponse_SqlCommandResult{},\n\t\t\t\t},\n\t\t\t},\n\t\t\tErr: nil,\n\t\t},\n\t\t{\n\t\t\tResp: &proto.ExecutePlanResponse{\n\t\t\t\tResponseType: &proto.ExecutePlanResponse_ResultComplete_{\n\t\t\t\t\tResultComplete: &proto.ExecutePlanResponse_ResultComplete{},\n\t\t\t\t},\n\t\t\t},\n\t\t\tErr: nil,\n\t\t},\n\t\t{\n\t\t\tErr: io.EOF,\n\t\t},\n\t}\n\n\ts := testutils.NewConnectServiceClientMock(&mocks.ProtoClient{\n\t\tRecvResponse: responses,\n\t}, nil, nil, t)\n\tc := client.NewSparkExecutorFromClient(s, nil, \"\")\n\n\tsession := &sparkSessionImpl{\n\t\tclient: c,\n\t}\n\tresp, err := session.Sql(ctx, query)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, resp)\n}\n\nfunc TestNewSessionBuilderCreatesASession(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := NewSessionBuilder().Remote(\"sc://connection\").Build(ctx)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, spark)\n}\n\nfunc TestNewSessionBuilderFailsIfConnectionStringIsInvalid(t *testing.T) {\n\tctx := context.Background()\n\tspark, err := NewSessionBuilder().Remote(\"invalid\").Build(ctx)\n\tassert.Error(t, err)\n\tassert.ErrorIs(t, err, sparkerrors.InvalidInputError)\n\tassert.Nil(t, spark)\n}\n\nfunc TestWriteResultStreamsArrowResultToCollector(t *testing.T) {\n\tctx := context.Background()\n\n\tarrowFields := []arrow.Field{\n\t\t{\n\t\t\tName: \"show_string\",\n\t\t\tType: &arrow.StringType{},\n\t\t},\n\t}\n\tarrowSchema := arrow.NewSchema(arrowFields, nil)\n\tvar buf bytes.Buffer\n\tarrowWriter := ipc.NewWriter(&buf, ipc.WithSchema(arrowSchema))\n\tdefer arrowWriter.Close()\n\n\talloc := memory.NewGoAllocator()\n\trecordBuilder := array.NewRecordBuilder(alloc, arrowSchema)\n\tdefer recordBuilder.Release()\n\n\trecordBuilder.Field(0).(*array.StringBuilder).Append(\"str1a\\nstr1b\")\n\trecordBuilder.Field(0).(*array.StringBuilder).Append(\"str2\")\n\n\trecord := recordBuilder.NewRecord()\n\tdefer record.Release()\n\n\terr := arrowWriter.Write(record)\n\trequire.Nil(t, err)\n\n\tquery := \"select * from bla\"\n\n\t// Create the responses:\n\tresponses := []*mocks.MockResponse{\n\t\t// The first stream of response is necessary for the SQL command.\n\t\t{\n\t\t\tResp: &proto.ExecutePlanResponse{\n\t\t\t\tResponseType: &proto.ExecutePlanResponse_SqlCommandResult_{\n\t\t\t\t\tSqlCommandResult: &proto.ExecutePlanResponse_SqlCommandResult{},\n\t\t\t\t},\n\t\t\t},\n\t\t\tErr: nil,\n\t\t},\n\t\t{\n\t\t\tResp: &proto.ExecutePlanResponse{\n\t\t\t\tResponseType: &proto.ExecutePlanResponse_ResultComplete_{\n\t\t\t\t\tResultComplete: &proto.ExecutePlanResponse_ResultComplete{},\n\t\t\t\t},\n\t\t\t},\n\t\t\tErr: nil,\n\t\t},\n\t\t{\n\t\t\tErr: io.EOF,\n\t\t},\n\t\t// The second stream of responses is for the actual execution\n\t\t{\n\t\t\tResp: &proto.ExecutePlanResponse{\n\t\t\t\tResponseType: &proto.ExecutePlanResponse_ArrowBatch_{\n\t\t\t\t\tArrowBatch: &proto.ExecutePlanResponse_ArrowBatch{\n\t\t\t\t\t\tRowCount: 2,\n\t\t\t\t\t\tData:     buf.Bytes(),\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tErr: io.EOF,\n\t\t},\n\t}\n\n\ts := testutils.NewConnectServiceClientMock(&mocks.ProtoClient{\n\t\tRecvResponse: responses,\n\t}, nil, nil, t)\n\tc := client.NewSparkExecutorFromClient(s, nil, \"\")\n\n\tsession := &sparkSessionImpl{\n\t\tclient: c,\n\t}\n\n\tresp, err := session.Sql(ctx, query)\n\tassert.NoError(t, err)\n\tassert.NotNil(t, resp)\n\tdf, err := resp.Repartition(ctx, 1, []string{\"1\"})\n\tassert.NoError(t, err)\n\trows, err := df.Collect(ctx)\n\tassert.NoError(t, err)\n\tvals := rows[1].Values()\n\tassert.NoError(t, err)\n\tassert.Equal(t, []any{\"str2\"}, vals)\n}\n"
  },
  {
    "path": "spark/sql/types/arrow.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"bytes\"\n\t\"fmt\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/arrow-go/v18/arrow/ipc\"\n\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\nfunc ReadArrowTableToRows(table arrow.Table) ([]Row, error) {\n\tresult := make([]Row, table.NumRows())\n\n\t// For each column in the table, read the data and convert it to an array of any.\n\tcols := make([][]any, table.NumCols())\n\tfor i := 0; i < int(table.NumCols()); i++ {\n\t\tchunkedColumn := table.Column(i).Data()\n\t\tcolumn, err := readChunkedColumn(chunkedColumn)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tcols[i] = column\n\t}\n\n\t// Create a list of field names for the rows.\n\tfieldNames := make([]string, table.NumCols())\n\tfor i, field := range table.Schema().Fields() {\n\t\tfieldNames[i] = field.Name\n\t}\n\n\t// Create the rows:\n\tfor i := 0; i < int(table.NumRows()); i++ {\n\t\trow := make([]any, table.NumCols())\n\t\tfor j := 0; j < int(table.NumCols()); j++ {\n\t\t\trow[j] = cols[j][i]\n\t\t}\n\t\tr := &rowImpl{\n\t\t\tvalues:  row,\n\t\t\toffsets: make(map[string]int),\n\t\t}\n\t\tfor j, fieldName := range fieldNames {\n\t\t\tr.offsets[fieldName] = j\n\t\t}\n\t\tresult[i] = r\n\t}\n\n\treturn result, nil\n}\n\nfunc readArrayData(t arrow.Type, data arrow.ArrayData) ([]any, error) {\n\tbuf := make([]any, 0)\n\t// Switch over the type t and append the values to buf.\n\tswitch t {\n\tcase arrow.BOOL:\n\t\tdata := array.NewBooleanData(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.INT8:\n\t\tdata := array.NewInt8Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.INT16:\n\t\tdata := array.NewInt16Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.INT32:\n\t\tdata := array.NewInt32Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.INT64:\n\t\tdata := array.NewInt64Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.FLOAT16:\n\t\tdata := array.NewFloat16Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.FLOAT32:\n\t\tdata := array.NewFloat32Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.FLOAT64:\n\t\tdata := array.NewFloat64Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.DECIMAL | arrow.DECIMAL128:\n\t\tdata := array.NewDecimal128Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.DECIMAL256:\n\t\tdata := array.NewDecimal256Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.STRING:\n\t\tdata := array.NewStringData(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.BINARY:\n\t\tdata := array.NewBinaryData(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.TIMESTAMP:\n\t\tdata := array.NewTimestampData(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.DATE64:\n\t\tdata := array.NewDate64Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.DATE32:\n\t\tdata := array.NewDate32Data(data)\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t} else {\n\t\t\t\tbuf = append(buf, data.Value(i))\n\t\t\t}\n\t\t}\n\tcase arrow.LIST:\n\t\tdata := array.NewListData(data)\n\t\tvalues := data.ListValues()\n\n\t\tres, err := readArrayData(values.DataType().ID(), values.Data())\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t\tcontinue\n\t\t\t}\n\t\t\tstart := data.Offsets()[i]\n\t\t\tend := data.Offsets()[i+1]\n\t\t\t// TODO: Unfortunately, this ends up being stored as a slice of slices of any. But not\n\t\t\t// the right type.\n\t\t\tbuf = append(buf, res[start:end])\n\t\t}\n\tcase arrow.MAP:\n\t\t// For maps the data is stored as a list of key value pairs. So to extract the maps,\n\t\t// we follow the same behavior as for lists but with two sub lists.\n\t\tdata := array.NewMapData(data)\n\t\tkeys := data.Keys()\n\t\tvalues := data.Items()\n\n\t\tkeyValues, err := readArrayData(keys.DataType().ID(), keys.Data())\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tvalueValues, err := readArrayData(values.DataType().ID(), values.Data())\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t\tcontinue\n\t\t\t}\n\t\t\ttmp := make(map[any]any)\n\n\t\t\tstart := data.Offsets()[i]\n\t\t\tend := data.Offsets()[i+1]\n\n\t\t\tk := keyValues[start:end]\n\t\t\tv := valueValues[start:end]\n\t\t\tfor j := 0; j < len(k); j++ {\n\t\t\t\ttmp[k[j]] = v[j]\n\t\t\t}\n\t\t\tbuf = append(buf, tmp)\n\t\t}\n\tcase arrow.STRUCT:\n\t\tdata := array.NewStructData(data)\n\t\tschema := data.DataType().(*arrow.StructType)\n\n\t\tfor i := 0; i < data.Len(); i++ {\n\t\t\tif data.IsNull(i) {\n\t\t\t\tbuf = append(buf, nil)\n\t\t\t\tcontinue\n\t\t\t}\n\t\t\ttmp := make(map[string]any)\n\n\t\t\tfor j := range data.NumField() {\n\t\t\t\tfield := data.Field(j)\n\t\t\t\tfieldValues, err := readArrayData(field.DataType().ID(), field.Data())\n\t\t\t\tif err != nil {\n\t\t\t\t\treturn nil, err\n\t\t\t\t}\n\t\t\t\ttmp[schema.Field(j).Name] = fieldValues[i]\n\t\t\t}\n\t\t\tbuf = append(buf, tmp)\n\t\t}\n\tdefault:\n\t\treturn nil, fmt.Errorf(\"unsupported arrow data type %s\", t.String())\n\t}\n\treturn buf, nil\n}\n\nfunc readChunkedColumn(chunked *arrow.Chunked) ([]any, error) {\n\tbuf := make([]any, 0)\n\tfor _, chunk := range chunked.Chunks() {\n\t\tdata := chunk.Data()\n\t\tt := data.DataType().ID()\n\t\tvalues, err := readArrayData(t, data)\n\t\tif err != nil {\n\t\t\treturn nil, err\n\t\t}\n\t\tbuf = append(buf, values...)\n\t}\n\treturn buf, nil\n}\n\nfunc ReadArrowBatchToRecord(data []byte, schema *StructType) (arrow.Record, error) {\n\treader := bytes.NewReader(data)\n\tarrowReader, err := ipc.NewReader(reader)\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to create arrow reader: %w\", err), sparkerrors.ReadError)\n\t}\n\tdefer arrowReader.Release()\n\n\trecord, err := arrowReader.Read()\n\trecord.Retain()\n\tif err != nil {\n\t\treturn nil, sparkerrors.WithType(fmt.Errorf(\"failed to read arrow record: %w\", err), sparkerrors.ReadError)\n\t}\n\treturn record, nil\n}\n\nfunc arrowStructToProtoStruct(schema *arrow.StructType) *proto.DataType_Struct_ {\n\tfields := make([]*proto.DataType_StructField, schema.NumFields())\n\tfor i, field := range schema.Fields() {\n\t\tfields[i] = &proto.DataType_StructField{\n\t\t\tName:     field.Name,\n\t\t\tDataType: ArrowTypeToProto(field.Type),\n\t\t}\n\t}\n\treturn &proto.DataType_Struct_{\n\t\tStruct: &proto.DataType_Struct{\n\t\t\tFields: fields,\n\t\t},\n\t}\n}\n\nfunc ArrowTypeToProto(dataType arrow.DataType) *proto.DataType {\n\tswitch dataType.ID() {\n\tcase arrow.BOOL:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Boolean_{}}\n\tcase arrow.INT8:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Byte_{}}\n\tcase arrow.INT16:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Short_{}}\n\tcase arrow.INT32:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Integer_{}}\n\tcase arrow.INT64:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Long_{}}\n\tcase arrow.FLOAT16:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Float_{}}\n\tcase arrow.FLOAT32:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Double_{}}\n\tcase arrow.FLOAT64:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Double_{}}\n\tcase arrow.DECIMAL | arrow.DECIMAL128:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Decimal_{}}\n\tcase arrow.DECIMAL256:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Decimal_{}}\n\tcase arrow.STRING:\n\t\treturn &proto.DataType{Kind: &proto.DataType_String_{}}\n\tcase arrow.BINARY:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Binary_{}}\n\tcase arrow.TIMESTAMP:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Timestamp_{}}\n\tcase arrow.DATE64:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Date_{}}\n\tcase arrow.LIST:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Array_{\n\t\t\tArray: &proto.DataType_Array{\n\t\t\t\tElementType: ArrowTypeToProto(dataType.(*arrow.ListType).Elem()),\n\t\t\t},\n\t\t}}\n\tcase arrow.STRUCT:\n\t\treturn &proto.DataType{Kind: arrowStructToProtoStruct(dataType.(*arrow.StructType))}\n\tdefault:\n\t\treturn &proto.DataType{Kind: &proto.DataType_Unparsed_{}}\n\t}\n}\n\nfunc ArrowSchemaToProto(schema *arrow.Schema) proto.DataType_Struct {\n\tfields := make([]*proto.DataType_StructField, schema.NumFields())\n\tfor i, field := range schema.Fields() {\n\t\tfields[i] = &proto.DataType_StructField{\n\t\t\tName:     field.Name,\n\t\t\tDataType: ArrowTypeToProto(field.Type),\n\t\t}\n\t}\n\treturn proto.DataType_Struct{\n\t\tFields: fields,\n\t}\n}\n"
  },
  {
    "path": "spark/sql/types/arrow_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types_test\n\nimport (\n\t\"bytes\"\n\t\"testing\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/array\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal128\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal256\"\n\t\"github.com/apache/arrow-go/v18/arrow/float16\"\n\t\"github.com/apache/arrow-go/v18/arrow/ipc\"\n\t\"github.com/apache/arrow-go/v18/arrow/memory\"\n\t\"github.com/stretchr/testify/assert\"\n\t\"github.com/stretchr/testify/require\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n)\n\nfunc TestShowArrowBatchData(t *testing.T) {\n\tarrowFields := []arrow.Field{\n\t\t{\n\t\t\tName: \"show_string\",\n\t\t\tType: &arrow.StringType{},\n\t\t},\n\t}\n\tarrowSchema := arrow.NewSchema(arrowFields, nil)\n\tvar buf bytes.Buffer\n\tarrowWriter := ipc.NewWriter(&buf, ipc.WithSchema(arrowSchema))\n\tdefer arrowWriter.Close()\n\n\talloc := memory.NewGoAllocator()\n\trecordBuilder := array.NewRecordBuilder(alloc, arrowSchema)\n\tdefer recordBuilder.Release()\n\n\trecordBuilder.Field(0).(*array.StringBuilder).Append(\"str1a\\nstr1b\")\n\trecordBuilder.Field(0).(*array.StringBuilder).Append(\"str2\")\n\n\trecord := recordBuilder.NewRecord()\n\tdefer record.Release()\n\n\terr := arrowWriter.Write(record)\n\trequire.Nil(t, err)\n\n\t// Convert the data\n\trecord, err = types.ReadArrowBatchToRecord(buf.Bytes(), nil)\n\trequire.NoError(t, err)\n\n\ttable := array.NewTableFromRecords(arrowSchema, []arrow.Record{record})\n\tvalues, err := types.ReadArrowTableToRows(table)\n\trequire.Nil(t, err)\n\tassert.Equal(t, 2, len(values))\n\tassert.Equal(t, []any{\"str1a\\nstr1b\"}, values[0].Values())\n\tassert.Equal(t, []any{\"str2\"}, values[1].Values())\n}\n\nfunc TestReadArrowRecord(t *testing.T) {\n\tarrowFields := []arrow.Field{\n\t\t{\n\t\t\tName: \"boolean_column\",\n\t\t\tType: &arrow.BooleanType{},\n\t\t},\n\t\t{\n\t\t\tName: \"int8_column\",\n\t\t\tType: &arrow.Int8Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"int16_column\",\n\t\t\tType: &arrow.Int16Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"int32_column\",\n\t\t\tType: &arrow.Int32Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"int64_column\",\n\t\t\tType: &arrow.Int64Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"float16_column\",\n\t\t\tType: &arrow.Float16Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"float32_column\",\n\t\t\tType: &arrow.Float32Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"float64_column\",\n\t\t\tType: &arrow.Float64Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"decimal128_column\",\n\t\t\tType: &arrow.Decimal128Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"decimal256_column\",\n\t\t\tType: &arrow.Decimal256Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"string_column\",\n\t\t\tType: &arrow.StringType{},\n\t\t},\n\t\t{\n\t\t\tName: \"binary_column\",\n\t\t\tType: &arrow.BinaryType{},\n\t\t},\n\t\t{\n\t\t\tName: \"timestamp_column\",\n\t\t\tType: &arrow.TimestampType{},\n\t\t},\n\t\t{\n\t\t\tName: \"date64_column\",\n\t\t\tType: &arrow.Date64Type{},\n\t\t},\n\t\t{\n\t\t\tName: \"array_int64_column\",\n\t\t\tType: arrow.ListOf(arrow.PrimitiveTypes.Int64),\n\t\t},\n\t\t{\n\t\t\tName: \"map_string_int32\",\n\t\t\tType: arrow.MapOf(arrow.BinaryTypes.String, arrow.PrimitiveTypes.Int32),\n\t\t},\n\t\t{\n\t\t\tName: \"struct\",\n\t\t\tType: arrow.StructOf(\n\t\t\t\tarrow.Field{Name: \"field1\", Type: arrow.PrimitiveTypes.Int32},\n\t\t\t\tarrow.Field{Name: \"field2\", Type: arrow.BinaryTypes.String},\n\t\t\t),\n\t\t},\n\t\t{\n\t\t\tName: \"nested_struct\",\n\t\t\tType: arrow.StructOf(\n\t\t\t\tarrow.Field{Name: \"field1\", Type: arrow.StructOf(\n\t\t\t\t\tarrow.Field{Name: \"nested_field1\", Type: arrow.PrimitiveTypes.Int32},\n\t\t\t\t\tarrow.Field{Name: \"nested_field2\", Type: arrow.BinaryTypes.String},\n\t\t\t\t)},\n\t\t\t),\n\t\t},\n\t}\n\tarrowSchema := arrow.NewSchema(arrowFields, nil)\n\tvar buf bytes.Buffer\n\tarrowWriter := ipc.NewWriter(&buf, ipc.WithSchema(arrowSchema))\n\tdefer arrowWriter.Close()\n\n\talloc := memory.NewGoAllocator()\n\trecordBuilder := array.NewRecordBuilder(alloc, arrowSchema)\n\tdefer recordBuilder.Release()\n\n\ti := 0\n\trecordBuilder.Field(i).(*array.BooleanBuilder).Append(false)\n\trecordBuilder.Field(i).(*array.BooleanBuilder).Append(true)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Int8Builder).Append(1)\n\trecordBuilder.Field(i).(*array.Int8Builder).Append(2)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Int16Builder).Append(10)\n\trecordBuilder.Field(i).(*array.Int16Builder).Append(20)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Int32Builder).Append(100)\n\trecordBuilder.Field(i).(*array.Int32Builder).Append(200)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Int64Builder).Append(1000)\n\trecordBuilder.Field(i).(*array.Int64Builder).Append(2000)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Float16Builder).Append(float16.New(10000.1))\n\trecordBuilder.Field(i).(*array.Float16Builder).Append(float16.New(20000.1))\n\n\ti++\n\trecordBuilder.Field(i).(*array.Float32Builder).Append(100000.1)\n\trecordBuilder.Field(i).(*array.Float32Builder).Append(200000.1)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Float64Builder).Append(1000000.1)\n\trecordBuilder.Field(i).(*array.Float64Builder).Append(2000000.1)\n\n\ti++\n\trecordBuilder.Field(i).(*array.Decimal128Builder).Append(decimal128.FromI64(10000000))\n\trecordBuilder.Field(i).(*array.Decimal128Builder).Append(decimal128.FromI64(20000000))\n\n\ti++\n\trecordBuilder.Field(i).(*array.Decimal256Builder).Append(decimal256.FromI64(100000000))\n\trecordBuilder.Field(i).(*array.Decimal256Builder).Append(decimal256.FromI64(200000000))\n\n\ti++\n\trecordBuilder.Field(i).(*array.StringBuilder).Append(\"str1\")\n\trecordBuilder.Field(i).(*array.StringBuilder).Append(\"str2\")\n\n\ti++\n\trecordBuilder.Field(i).(*array.BinaryBuilder).Append([]byte(\"bytes1\"))\n\trecordBuilder.Field(i).(*array.BinaryBuilder).Append([]byte(\"bytes2\"))\n\n\ti++\n\trecordBuilder.Field(i).(*array.TimestampBuilder).Append(arrow.Timestamp(1686981953115000))\n\trecordBuilder.Field(i).(*array.TimestampBuilder).Append(arrow.Timestamp(1686981953116000))\n\n\ti++\n\trecordBuilder.Field(i).(*array.Date64Builder).Append(arrow.Date64(1686981953117000))\n\trecordBuilder.Field(i).(*array.Date64Builder).Append(arrow.Date64(1686981953118000))\n\n\ti++\n\tlb := recordBuilder.Field(i).(*array.ListBuilder)\n\tlb.Append(true)\n\tlb.ValueBuilder().(*array.Int64Builder).Append(1)\n\tlb.ValueBuilder().(*array.Int64Builder).Append(-999231)\n\n\tlb.Append(true)\n\tlb.ValueBuilder().(*array.Int64Builder).Append(1)\n\tlb.ValueBuilder().(*array.Int64Builder).Append(2)\n\tlb.ValueBuilder().(*array.Int64Builder).Append(3)\n\n\ti++\n\tmb := recordBuilder.Field(i).(*array.MapBuilder)\n\tmb.Append(true)\n\tmb.KeyBuilder().(*array.StringBuilder).Append(\"key1\")\n\tmb.ItemBuilder().(*array.Int32Builder).Append(1)\n\n\tmb.Append(true)\n\tmb.KeyBuilder().(*array.StringBuilder).Append(\"key2\")\n\tmb.ItemBuilder().(*array.Int32Builder).Append(2)\n\n\ti++\n\tsb := recordBuilder.Field(i).(*array.StructBuilder)\n\tsb.Append(true)\n\tsb.FieldBuilder(0).(*array.Int32Builder).Append(1)\n\tsb.FieldBuilder(1).(*array.StringBuilder).Append(\"str1\")\n\n\tsb.Append(true)\n\tsb.FieldBuilder(0).(*array.Int32Builder).Append(2)\n\tsb.FieldBuilder(1).(*array.StringBuilder).Append(\"str2\")\n\n\ti++\n\tsb = recordBuilder.Field(i).(*array.StructBuilder)\n\tsb.Append(true)\n\tnsb := sb.FieldBuilder(0).(*array.StructBuilder)\n\tnsb.Append(true)\n\tnsb.FieldBuilder(0).(*array.Int32Builder).Append(1)\n\tnsb.FieldBuilder(1).(*array.StringBuilder).Append(\"str1_nested\")\n\n\tsb.Append(true)\n\tnsb = sb.FieldBuilder(0).(*array.StructBuilder)\n\tnsb.Append(true)\n\tnsb.FieldBuilder(0).(*array.Int32Builder).Append(2)\n\tnsb.FieldBuilder(1).(*array.StringBuilder).Append(\"str2_nested\")\n\n\trecord := recordBuilder.NewRecord()\n\tdefer record.Release()\n\n\ttable := array.NewTableFromRecords(arrowSchema, []arrow.Record{record})\n\tvalues, err := types.ReadArrowTableToRows(table)\n\trequire.Nil(t, err)\n\tassert.Equal(t, 2, len(values))\n\tassert.Equal(t, []any{\n\t\tfalse, int8(1), int16(10), int32(100), int64(1000),\n\t\tfloat16.New(10000.1), float32(100000.1), 1000000.1,\n\t\tdecimal128.FromI64(10000000), decimal256.FromI64(100000000),\n\t\t\"str1\", []byte(\"bytes1\"),\n\t\tarrow.Timestamp(1686981953115000), arrow.Date64(1686981953117000),\n\t\t[]any{int64(1), int64(-999231)},\n\t\tmap[any]any{\"key1\": int32(1)},\n\t\tmap[string]any{\"field1\": int32(1), \"field2\": \"str1\"},\n\t\tmap[string]any{\n\t\t\t\"field1\": map[string]any{\n\t\t\t\t\"nested_field1\": int32(1),\n\t\t\t\t\"nested_field2\": \"str1_nested\",\n\t\t\t},\n\t\t},\n\t},\n\t\tvalues[0].Values())\n\tassert.Equal(t, []any{\n\t\ttrue, int8(2), int16(20), int32(200), int64(2000),\n\t\tfloat16.New(20000.1), float32(200000.1), 2000000.1,\n\t\tdecimal128.FromI64(20000000), decimal256.FromI64(200000000),\n\t\t\"str2\", []byte(\"bytes2\"),\n\t\tarrow.Timestamp(1686981953116000), arrow.Date64(1686981953118000),\n\t\t[]any{int64(1), int64(2), int64(3)},\n\t\tmap[any]any{\"key2\": int32(2)},\n\t\tmap[string]any{\"field1\": int32(2), \"field2\": \"str2\"},\n\t\tmap[string]any{\n\t\t\t\"field1\": map[string]any{\n\t\t\t\t\"nested_field1\": int32(2),\n\t\t\t\t\"nested_field2\": \"str2_nested\",\n\t\t\t},\n\t\t},\n\t},\n\t\tvalues[1].Values())\n}\n\nfunc TestReadArrowRecord_UnsupportedType(t *testing.T) {\n\tarrowFields := []arrow.Field{\n\t\t{\n\t\t\tName: \"unsupported_type_column\",\n\t\t\tType: &arrow.MonthIntervalType{},\n\t\t},\n\t}\n\tarrowSchema := arrow.NewSchema(arrowFields, nil)\n\tvar buf bytes.Buffer\n\tarrowWriter := ipc.NewWriter(&buf, ipc.WithSchema(arrowSchema))\n\tdefer arrowWriter.Close()\n\n\talloc := memory.NewGoAllocator()\n\trecordBuilder := array.NewRecordBuilder(alloc, arrowSchema)\n\tdefer recordBuilder.Release()\n\n\trecordBuilder.Field(0).(*array.MonthIntervalBuilder).Append(1)\n\n\trecord := recordBuilder.NewRecord()\n\tdefer record.Release()\n\n\ttable := array.NewTableFromRecords(arrowSchema, []arrow.Record{record})\n\t_, err := types.ReadArrowTableToRows(table)\n\trequire.NotNil(t, err)\n}\n\nfunc TestConvertProtoDataTypeToDataType(t *testing.T) {\n\tbooleanDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Boolean_{},\n\t}\n\tassert.Equal(t, \"Boolean\", types.ConvertProtoDataTypeToDataType(booleanDataType).TypeName())\n\n\tbyteDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Byte_{},\n\t}\n\tassert.Equal(t, \"Byte\", types.ConvertProtoDataTypeToDataType(byteDataType).TypeName())\n\n\tshortDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Short_{},\n\t}\n\tassert.Equal(t, \"Short\", types.ConvertProtoDataTypeToDataType(shortDataType).TypeName())\n\n\tintegerDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Integer_{},\n\t}\n\tassert.Equal(t, \"Integer\", types.ConvertProtoDataTypeToDataType(integerDataType).TypeName())\n\n\tlongDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Long_{},\n\t}\n\tassert.Equal(t, \"Long\", types.ConvertProtoDataTypeToDataType(longDataType).TypeName())\n\n\tfloatDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Float_{},\n\t}\n\tassert.Equal(t, \"Float\", types.ConvertProtoDataTypeToDataType(floatDataType).TypeName())\n\n\tdoubleDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Double_{},\n\t}\n\tassert.Equal(t, \"Double\", types.ConvertProtoDataTypeToDataType(doubleDataType).TypeName())\n\n\tdecimalDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Decimal_{},\n\t}\n\tassert.Equal(t, \"Decimal\", types.ConvertProtoDataTypeToDataType(decimalDataType).TypeName())\n\n\tstringDataType := &proto.DataType{\n\t\tKind: &proto.DataType_String_{},\n\t}\n\tassert.Equal(t, \"String\", types.ConvertProtoDataTypeToDataType(stringDataType).TypeName())\n\n\tbinaryDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Binary_{},\n\t}\n\tassert.Equal(t, \"Binary\", types.ConvertProtoDataTypeToDataType(binaryDataType).TypeName())\n\n\ttimestampDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Timestamp_{},\n\t}\n\tassert.Equal(t, \"Timestamp\", types.ConvertProtoDataTypeToDataType(timestampDataType).TypeName())\n\n\ttimestampNtzDataType := &proto.DataType{\n\t\tKind: &proto.DataType_TimestampNtz{},\n\t}\n\tassert.Equal(t, \"TimestampNtz\", types.ConvertProtoDataTypeToDataType(timestampNtzDataType).TypeName())\n\n\tdateDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Date_{},\n\t}\n\tassert.Equal(t, \"Date\", types.ConvertProtoDataTypeToDataType(dateDataType).TypeName())\n\n\tarrayDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Array_{\n\t\t\tArray: &proto.DataType_Array{\n\t\t\t\tElementType: &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t},\n\t\t},\n\t}\n\tassert.Equal(t, \"Array<Integer>\", types.ConvertProtoDataTypeToDataType(arrayDataType).TypeName())\n\n\tmapDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Map_{\n\t\t\tMap: &proto.DataType_Map{\n\t\t\t\tKeyType:           &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\tValueType:         &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\tValueContainsNull: true,\n\t\t\t},\n\t\t},\n\t}\n\tassert.Equal(t, \"Map<String,Integer>\", types.ConvertProtoDataTypeToDataType(mapDataType).TypeName())\n\n\tstructDataType := &proto.DataType{\n\t\tKind: &proto.DataType_Struct_{\n\t\t\tStruct: &proto.DataType_Struct{\n\t\t\t\tFields: []*proto.DataType_StructField{\n\t\t\t\t\t{Name: \"field1\", DataType: &proto.DataType{Kind: &proto.DataType_Integer_{}}},\n\t\t\t\t\t{Name: \"field2\", DataType: &proto.DataType{Kind: &proto.DataType_String_{}}},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tassert.Equal(t, \"structtype\", types.ConvertProtoDataTypeToDataType(structDataType).TypeName())\n}\n\nfunc TestConvertProtoDataTypeToDataType_UnsupportedType(t *testing.T) {\n\tunsupportedDataType := &proto.DataType{\n\t\tKind: &proto.DataType_YearMonthInterval_{},\n\t}\n\tassert.Equal(t, \"Unsupported\", types.ConvertProtoDataTypeToDataType(unsupportedDataType).TypeName())\n}\n"
  },
  {
    "path": "spark/sql/types/builtin.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"context\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n)\n\ntype LiteralType interface {\n\tToProto(ctx context.Context) (*proto.Expression, error)\n}\n\ntype NumericLiteral interface {\n\tLiteralType\n\t// marker method for compile time safety.\n\tisNumericLiteral()\n}\n\ntype PrimitiveTypeLiteral interface {\n\tLiteralType\n\tisPrimitiveTypeLiteral()\n}\n\ntype Int8 int8\n\nfunc (t Int8) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Byte{Byte: int32(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Int8) isNumericLiteral() {}\n\nfunc (t Int8) isPrimitiveTypeLiteral() {}\n\ntype Int16 int16\n\nfunc (t Int16) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Short{Short: int32(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Int16) isNumericLiteral() {}\n\nfunc (t Int16) isPrimitiveTypeLiteral() {}\n\ntype Int32 int32\n\nfunc (t Int32) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Integer{Integer: int32(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Int32) isNumericLiteral() {}\n\nfunc (t Int32) isPrimitiveTypeLiteral() {}\n\ntype Int64 int64\n\nfunc (t Int64) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Long{Long: int64(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Int64) isNumericLiteral() {}\n\nfunc (t Int64) isPrimitiveTypeLiteral() {}\n\ntype Int int\n\nfunc (t Int) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn Int64(t).ToProto(ctx)\n}\n\nfunc (t Int) isNumericLiteral() {}\n\nfunc (t Int) isPrimitiveTypeLiteral() {}\n\ntype Float32 float32\n\nfunc (t Float32) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Float{Float: float32(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Float32) isNumericLiteral() {}\n\nfunc (t Float32) isPrimitiveTypeLiteral() {}\n\ntype Float64 float64\n\nfunc (t Float64) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Double{Double: float64(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Float64) isNumericLiteral() {}\n\nfunc (t Float64) isPrimitiveTypeLiteral() {}\n\ntype String string\n\nfunc (t String) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_String_{String_: string(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t String) isPrimitiveTypeLiteral() {}\n\ntype Boolean bool\n\nfunc (t Boolean) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Boolean{Boolean: bool(t)},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\nfunc (t Boolean) isPrimitiveTypeLiteral() {}\n\ntype Binary []byte\n\nfunc (t Binary) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Binary{Binary: t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype Int8NilType struct{}\n\nvar Int8Nil = Int8NilType{}\n\nfunc (t Int8NilType) isNumericLiteral() {}\n\nfunc (t Int8NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Int8NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Byte_{\n\t\t\t\t\t\t\tByte: &proto.DataType_Byte{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype Int16NilType struct{}\n\nvar Int16Nil = Int16NilType{}\n\nfunc (t Int16NilType) isNumericLiteral() {}\n\nfunc (t Int16NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Int16NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Short_{\n\t\t\t\t\t\t\tShort: &proto.DataType_Short{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype Int32NilType struct{}\n\nvar Int32Nil = Int32NilType{}\n\nfunc (t Int32NilType) isNumericLiteral() {}\n\nfunc (t Int32NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Int32NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Integer_{\n\t\t\t\t\t\t\tInteger: &proto.DataType_Integer{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype Int64NilType struct{}\n\nvar Int64Nil = Int64NilType{}\n\nfunc (t Int64NilType) isNumericLiteral() {}\n\nfunc (t Int64NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Int64NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Long_{\n\t\t\t\t\t\t\tLong: &proto.DataType_Long{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype IntNilType struct{}\n\nvar IntNil = IntNilType{}\n\nfunc (t IntNilType) isNumericLiteral() {}\n\nfunc (t IntNilType) isPrimitiveTypeLiteral() {}\n\nfunc (t IntNilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn Int64NilType{}.ToProto(ctx)\n}\n\ntype Float32NilType struct{}\n\nvar Float32Nil = Float32NilType{}\n\nfunc (t Float32NilType) isNumericLiteral() {}\n\nfunc (t Float32NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Float32NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Float_{\n\t\t\t\t\t\t\tFloat: &proto.DataType_Float{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype Float64NilType struct{}\n\nvar Float64Nil = Float64NilType{}\n\nfunc (t Float64NilType) isNumericLiteral() {}\n\nfunc (t Float64NilType) isPrimitiveTypeLiteral() {}\n\nfunc (t Float64NilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Double_{\n\t\t\t\t\t\t\tDouble: &proto.DataType_Double{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype StringNilType struct{}\n\nvar StringNil = StringNilType{}\n\nfunc (t StringNilType) isPrimitiveTypeLiteral() {}\n\nfunc (t StringNilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_String_{\n\t\t\t\t\t\t\tString_: &proto.DataType_String{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype BooleanNilType struct{}\n\nvar BooleanNil = BooleanNilType{}\n\nfunc (t BooleanNilType) isPrimitiveTypeLiteral() {}\n\nfunc (t BooleanNilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Boolean_{\n\t\t\t\t\t\t\tBoolean: &proto.DataType_Boolean{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n\ntype BinaryNilType struct{}\n\nvar BinaryNil = BinaryNilType{}\n\nfunc (t BinaryNilType) ToProto(ctx context.Context) (*proto.Expression, error) {\n\treturn &proto.Expression{\n\t\tExprType: &proto.Expression_Literal_{\n\t\t\tLiteral: &proto.Expression_Literal{\n\t\t\t\tLiteralType: &proto.Expression_Literal_Null{\n\t\t\t\t\tNull: &proto.DataType{\n\t\t\t\t\t\tKind: &proto.DataType_Binary_{\n\t\t\t\t\t\t\tBinary: &proto.DataType_Binary{},\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}, nil\n}\n"
  },
  {
    "path": "spark/sql/types/builtin_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"context\"\n\t\"testing\"\n\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestBuiltinTypes(t *testing.T) {\n\tp, err := Int8(1).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetByte(), int32(1))\n\n\tp, err = Int16(1).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetShort(), int32(1))\n\n\tp, err = Int32(1).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetInteger(), int32(1))\n\n\tp, err = Int64(1).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetLong(), int64(1))\n\n\tp, err = Float32(1.0).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetFloat(), float32(1.0))\n\n\tp, err = Float64(1.0).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetDouble(), float64(1.0))\n\n\tp, err = String(\"1\").ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetString_(), \"1\")\n\n\tp, err = Boolean(true).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetBoolean(), true)\n\n\tp, err = Binary([]byte{1}).ToProto(context.TODO())\n\tassert.NoError(t, err)\n\tassert.Equal(t, p.GetLiteral().GetBinary(), []byte{1})\n}\n\nfunc testMe(n NumericLiteral) bool {\n\treturn true\n}\n\nfunc testPrimitive(p PrimitiveTypeLiteral) bool {\n\treturn true\n}\n\nfunc TestNumericTypes(t *testing.T) {\n\tassert.True(t, testMe(Int8(1)))\n\tassert.True(t, testMe(Int16(1)))\n\tassert.True(t, testMe(Int32(1)))\n\tassert.True(t, testMe(Int64(1)))\n\tassert.True(t, testMe(Float32(1.0)))\n\tassert.True(t, testMe(Float64(1.0)))\n\n\tassert.True(t, testPrimitive(String(\"a\")))\n\tassert.True(t, testPrimitive(Boolean(true)))\n\tassert.True(t, testPrimitive(Int16(1)))\n}\n"
  },
  {
    "path": "spark/sql/types/conversion.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"errors\"\n\n\t\"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sparkerrors\"\n)\n\nfunc ConvertProtoDataTypeToStructType(input *generated.DataType) (*StructType, error) {\n\tdataTypeStruct := input.GetStruct()\n\tif dataTypeStruct == nil {\n\t\treturn nil, sparkerrors.WithType(errors.New(\"dataType.GetStruct() is nil\"), sparkerrors.InvalidInputError)\n\t}\n\treturn &StructType{\n\t\tFields: ConvertProtoStructFields(dataTypeStruct.Fields),\n\t}, nil\n}\n\nfunc ConvertProtoStructFields(input []*generated.DataType_StructField) []StructField {\n\tresult := make([]StructField, len(input))\n\tfor i, f := range input {\n\t\tresult[i] = ConvertProtoStructField(f)\n\t}\n\treturn result\n}\n\nfunc ConvertProtoStructField(field *generated.DataType_StructField) StructField {\n\treturn StructField{\n\t\tName:     field.Name,\n\t\tDataType: ConvertProtoDataTypeToDataType(field.DataType),\n\t\tNullable: field.Nullable,\n\t\tMetadata: field.Metadata,\n\t}\n}\n\n// ConvertProtoDataTypeToDataType converts protobuf data type to Spark connect sql data type\nfunc ConvertProtoDataTypeToDataType(input *generated.DataType) DataType {\n\tswitch v := input.GetKind().(type) {\n\tcase *generated.DataType_Boolean_:\n\t\treturn BooleanType{}\n\tcase *generated.DataType_Byte_:\n\t\treturn ByteType{}\n\tcase *generated.DataType_Short_:\n\t\treturn ShortType{}\n\tcase *generated.DataType_Integer_:\n\t\treturn IntegerType{}\n\tcase *generated.DataType_Long_:\n\t\treturn LongType{}\n\tcase *generated.DataType_Float_:\n\t\treturn FloatType{}\n\tcase *generated.DataType_Double_:\n\t\treturn DoubleType{}\n\tcase *generated.DataType_Decimal_:\n\t\treturn DecimalType{}\n\tcase *generated.DataType_String_:\n\t\treturn StringType{}\n\tcase *generated.DataType_Binary_:\n\t\treturn BinaryType{}\n\tcase *generated.DataType_Timestamp_:\n\t\treturn TimestampType{}\n\tcase *generated.DataType_TimestampNtz:\n\t\treturn TimestampNtzType{}\n\tcase *generated.DataType_Date_:\n\t\treturn DateType{}\n\tcase *generated.DataType_Array_:\n\t\tnestedType := ConvertProtoDataTypeToDataType(input.GetArray().ElementType)\n\t\tcontainsNull := input.GetArray().ContainsNull\n\t\treturn ArrayType{\n\t\t\tElementType:  nestedType,\n\t\t\tContainsNull: containsNull,\n\t\t}\n\tcase *generated.DataType_Map_:\n\t\tkeyType := ConvertProtoDataTypeToDataType(input.GetMap().KeyType)\n\t\tvalueType := ConvertProtoDataTypeToDataType(input.GetMap().ValueType)\n\t\tvalueContainsNull := input.GetMap().ValueContainsNull\n\t\treturn MapType{\n\t\t\tKeyType:           keyType,\n\t\t\tValueType:         valueType,\n\t\t\tValueContainsNull: valueContainsNull,\n\t\t}\n\tcase *generated.DataType_Struct_:\n\t\tfields := ConvertProtoStructFields(input.GetStruct().Fields)\n\t\treturn *StructOf(fields...)\n\tdefault:\n\t\treturn UnsupportedType{\n\t\t\tTypeInfo: v,\n\t\t}\n\t}\n}\n"
  },
  {
    "path": "spark/sql/types/conversion_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types_test\n\nimport (\n\t\"testing\"\n\n\tproto \"github.com/apache/spark-connect-go/internal/generated\"\n\t\"github.com/apache/spark-connect-go/spark/sql/types\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestConvertProtoStructFieldSupported(t *testing.T) {\n\tprotoType := &proto.DataType{Kind: &proto.DataType_Integer_{}}\n\tstructField := &proto.DataType_StructField{\n\t\tName:     \"test\",\n\t\tDataType: protoType,\n\t\tNullable: true,\n\t}\n\n\tdt := types.ConvertProtoStructField(structField)\n\tassert.Equal(t, \"test\", dt.Name)\n\tassert.IsType(t, types.IntegerType{}, dt.DataType)\n}\n\nfunc TestConvertProtoStructFieldUnsupported(t *testing.T) {\n\tprotoType := &proto.DataType{Kind: &proto.DataType_CalendarInterval_{}}\n\tstructField := &proto.DataType_StructField{\n\t\tName:     \"test\",\n\t\tDataType: protoType,\n\t\tNullable: true,\n\t}\n\n\tdt := types.ConvertProtoStructField(structField)\n\tassert.Equal(t, \"test\", dt.Name)\n\tassert.IsType(t, types.UnsupportedType{}, dt.DataType)\n}\n\nfunc TestConvertProtoStructToGoStruct(t *testing.T) {\n\tprotoType := &proto.DataType{\n\t\tKind: &proto.DataType_Struct_{\n\t\t\tStruct: &proto.DataType_Struct{\n\t\t\t\tFields: []*proto.DataType_StructField{\n\t\t\t\t\t{\n\t\t\t\t\t\tName:     \"test\",\n\t\t\t\t\t\tDataType: &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\t\t\tNullable: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\tstructType, err := types.ConvertProtoDataTypeToStructType(protoType)\n\tassert.NoError(t, err)\n\tassert.Equal(t, 1, len(structType.Fields))\n\tassert.Equal(t, \"test\", structType.Fields[0].Name)\n\tassert.IsType(t, types.IntegerType{}, structType.Fields[0].DataType)\n\n\t// Check for input type that is not a struct type and it returns an error.\n\tprotoType = &proto.DataType{Kind: &proto.DataType_Integer_{}}\n\t_, err = types.ConvertProtoDataTypeToStructType(protoType)\n\tassert.Error(t, err)\n}\n\nfunc TestConvertProtoArrayType(t *testing.T) {\n\ttests := []struct {\n\t\tname         string\n\t\tprotoType    *proto.DataType\n\t\texpectedType types.DataType\n\t\texpectedName string\n\t}{\n\t\t{\n\t\t\tname: \"Array of integers\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\tElementType:  &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\t\t\tContainsNull: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.ArrayType{\n\t\t\t\tElementType:  types.INTEGER,\n\t\t\t\tContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Array<Integer>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Array of strings without nulls\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\tElementType:  &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\tContainsNull: false,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.ArrayType{\n\t\t\t\tElementType:  types.STRING,\n\t\t\t\tContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Array<String>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Nested array\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\tElementType: &proto.DataType{\n\t\t\t\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\t\t\t\tElementType:  &proto.DataType{Kind: &proto.DataType_Double_{}},\n\t\t\t\t\t\t\t\t\tContainsNull: false,\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t\tContainsNull: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.ArrayType{\n\t\t\t\tElementType: types.ArrayType{\n\t\t\t\t\tElementType:  types.DOUBLE,\n\t\t\t\t\tContainsNull: false,\n\t\t\t\t},\n\t\t\t\tContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Array<Array<Double>>\",\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tconvertedType := types.ConvertProtoDataTypeToDataType(tt.protoType)\n\t\t\tassert.Equal(t, tt.expectedType, convertedType)\n\t\t\tassert.Equal(t, tt.expectedName, convertedType.TypeName())\n\t\t})\n\t}\n}\n\nfunc TestConvertProtoMapType(t *testing.T) {\n\ttests := []struct {\n\t\tname         string\n\t\tprotoType    *proto.DataType\n\t\texpectedType types.DataType\n\t\texpectedName string\n\t}{\n\t\t{\n\t\t\tname: \"Map of string to integer\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Map_{\n\t\t\t\t\tMap: &proto.DataType_Map{\n\t\t\t\t\t\tKeyType:           &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\tValueType:         &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\t\t\tValueContainsNull: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.MapType{\n\t\t\t\tKeyType:           types.STRING,\n\t\t\t\tValueType:         types.INTEGER,\n\t\t\t\tValueContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Map<String,Integer>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Map with array values\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Map_{\n\t\t\t\t\tMap: &proto.DataType_Map{\n\t\t\t\t\t\tKeyType: &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\t\t\tValueType: &proto.DataType{\n\t\t\t\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\t\t\t\tElementType:  &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\t\t\t\tContainsNull: true,\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t\tValueContainsNull: false,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.MapType{\n\t\t\t\tKeyType: types.INTEGER,\n\t\t\t\tValueType: types.ArrayType{\n\t\t\t\t\tElementType:  types.STRING,\n\t\t\t\t\tContainsNull: true,\n\t\t\t\t},\n\t\t\t\tValueContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Map<Integer,Array<String>>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Nested map\",\n\t\t\tprotoType: &proto.DataType{\n\t\t\t\tKind: &proto.DataType_Map_{\n\t\t\t\t\tMap: &proto.DataType_Map{\n\t\t\t\t\t\tKeyType: &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\tValueType: &proto.DataType{\n\t\t\t\t\t\t\tKind: &proto.DataType_Map_{\n\t\t\t\t\t\t\t\tMap: &proto.DataType_Map{\n\t\t\t\t\t\t\t\t\tKeyType:           &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\t\t\t\tValueType:         &proto.DataType{Kind: &proto.DataType_Double_{}},\n\t\t\t\t\t\t\t\t\tValueContainsNull: false,\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t\tValueContainsNull: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t\texpectedType: types.MapType{\n\t\t\t\tKeyType: types.STRING,\n\t\t\t\tValueType: types.MapType{\n\t\t\t\t\tKeyType:           types.STRING,\n\t\t\t\t\tValueType:         types.DOUBLE,\n\t\t\t\t\tValueContainsNull: false,\n\t\t\t\t},\n\t\t\t\tValueContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Map<String,Map<String,Double>>\",\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tconvertedType := types.ConvertProtoDataTypeToDataType(tt.protoType)\n\t\t\tassert.Equal(t, tt.expectedType, convertedType)\n\t\t\tassert.Equal(t, tt.expectedName, convertedType.TypeName())\n\t\t})\n\t}\n}\n\nfunc TestConvertComplexNestedTypes(t *testing.T) {\n\t// Test a complex nested structure: Struct containing Array<Map<String, Integer>>\n\tprotoType := &proto.DataType{\n\t\tKind: &proto.DataType_Struct_{\n\t\t\tStruct: &proto.DataType_Struct{\n\t\t\t\tFields: []*proto.DataType_StructField{\n\t\t\t\t\t{\n\t\t\t\t\t\tName: \"complex_field\",\n\t\t\t\t\t\tDataType: &proto.DataType{\n\t\t\t\t\t\t\tKind: &proto.DataType_Array_{\n\t\t\t\t\t\t\t\tArray: &proto.DataType_Array{\n\t\t\t\t\t\t\t\t\tElementType: &proto.DataType{\n\t\t\t\t\t\t\t\t\t\tKind: &proto.DataType_Map_{\n\t\t\t\t\t\t\t\t\t\t\tMap: &proto.DataType_Map{\n\t\t\t\t\t\t\t\t\t\t\t\tKeyType:           &proto.DataType{Kind: &proto.DataType_String_{}},\n\t\t\t\t\t\t\t\t\t\t\t\tValueType:         &proto.DataType{Kind: &proto.DataType_Integer_{}},\n\t\t\t\t\t\t\t\t\t\t\t\tValueContainsNull: true,\n\t\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t\t\tContainsNull: false,\n\t\t\t\t\t\t\t\t},\n\t\t\t\t\t\t\t},\n\t\t\t\t\t\t},\n\t\t\t\t\t\tNullable: true,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t},\n\t\t},\n\t}\n\n\tconvertedType := types.ConvertProtoDataTypeToDataType(protoType)\n\tstructType, ok := convertedType.(types.StructType)\n\tassert.True(t, ok)\n\tassert.Equal(t, 1, len(structType.Fields))\n\tassert.Equal(t, \"complex_field\", structType.Fields[0].Name)\n\n\tarrayType, ok := structType.Fields[0].DataType.(types.ArrayType)\n\tassert.True(t, ok)\n\tassert.False(t, arrayType.ContainsNull)\n\n\tmapType, ok := arrayType.ElementType.(types.MapType)\n\tassert.True(t, ok)\n\tassert.Equal(t, types.STRING, mapType.KeyType)\n\tassert.Equal(t, types.INTEGER, mapType.ValueType)\n\tassert.True(t, mapType.ValueContainsNull)\n}\n"
  },
  {
    "path": "spark/sql/types/datatype.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"fmt\"\n\t\"strings\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n)\n\ntype DataType interface {\n\tTypeName() string\n\tIsNumeric() bool\n\tToArrowType() arrow.DataType\n}\n\ntype BooleanType struct{}\n\nfunc (t BooleanType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t BooleanType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t BooleanType) ToArrowType() arrow.DataType {\n\treturn arrow.FixedWidthTypes.Boolean\n}\n\ntype ByteType struct{}\n\nfunc (t ByteType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t ByteType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Int8\n}\n\nfunc (t ByteType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\ntype ShortType struct{}\n\nfunc (t ShortType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t ShortType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t ShortType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Int16\n}\n\ntype IntegerType struct{}\n\nfunc (t IntegerType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t IntegerType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t IntegerType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Int32\n}\n\ntype LongType struct{}\n\nfunc (t LongType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t LongType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t LongType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Int64\n}\n\ntype FloatType struct{}\n\nfunc (t FloatType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t FloatType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t FloatType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Float32\n}\n\ntype DoubleType struct{}\n\nfunc (t DoubleType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t DoubleType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t DoubleType) ToArrowType() arrow.DataType {\n\treturn arrow.PrimitiveTypes.Float64\n}\n\ntype DecimalType struct {\n\tPrecision int32\n\tScale     int32\n}\n\nfunc (t DecimalType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t DecimalType) IsNumeric() bool {\n\treturn true\n}\n\nfunc (t DecimalType) ToArrowType() arrow.DataType {\n\treturn &arrow.Decimal128Type{\n\t\tPrecision: t.Precision,\n\t\tScale:     t.Scale,\n\t}\n}\n\ntype StringType struct{}\n\nfunc (t StringType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t StringType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t StringType) ToArrowType() arrow.DataType {\n\treturn arrow.BinaryTypes.String\n}\n\ntype BinaryType struct{}\n\nfunc (t BinaryType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t BinaryType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t BinaryType) ToArrowType() arrow.DataType {\n\treturn arrow.BinaryTypes.Binary\n}\n\ntype TimestampType struct{}\n\nfunc (t TimestampType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t TimestampType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t TimestampType) ToArrowType() arrow.DataType {\n\treturn arrow.FixedWidthTypes.Timestamp_ms\n}\n\ntype TimestampNtzType struct{}\n\nfunc (t TimestampNtzType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t TimestampNtzType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t TimestampNtzType) ToArrowType() arrow.DataType {\n\treturn arrow.FixedWidthTypes.Timestamp_ns\n}\n\ntype DateType struct{}\n\nfunc (t DateType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t DateType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t DateType) ToArrowType() arrow.DataType {\n\treturn arrow.FixedWidthTypes.Date32\n}\n\ntype ArrayType struct {\n\tElementType  DataType\n\tContainsNull bool\n}\n\nfunc (t ArrayType) TypeName() string {\n\treturn fmt.Sprintf(\"Array<%s>\", t.ElementType.TypeName())\n}\n\nfunc (t ArrayType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t ArrayType) ToArrowType() arrow.DataType {\n\treturn arrow.ListOf(t.ElementType.ToArrowType())\n}\n\ntype MapType struct {\n\tKeyType           DataType\n\tValueType         DataType\n\tValueContainsNull bool\n}\n\nfunc (t MapType) TypeName() string {\n\treturn fmt.Sprintf(\"Map<%s,%s>\", t.KeyType.TypeName(), t.ValueType.TypeName())\n}\n\nfunc (t MapType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t MapType) ToArrowType() arrow.DataType {\n\t// TODO: assert that ValueContainsNull is true because it indicates\n\t// nullability of the map type\n\treturn arrow.MapOf(t.KeyType.ToArrowType(), t.ValueType.ToArrowType())\n}\n\ntype UnsupportedType struct {\n\tTypeInfo any\n}\n\nfunc (t UnsupportedType) TypeName() string {\n\treturn getDataTypeName(t)\n}\n\nfunc (t UnsupportedType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t UnsupportedType) ToArrowType() arrow.DataType {\n\treturn nil\n}\n\nfunc getDataTypeName(dataType DataType) string {\n\ttypeName := fmt.Sprintf(\"%T\", dataType)\n\tnonQualifiedTypeName := strings.Split(typeName, \".\")[1]\n\treturn strings.TrimSuffix(nonQualifiedTypeName, \"Type\")\n}\n\nfunc MakeArrayType(elementType DataType, containsNull bool) ArrayType {\n\treturn ArrayType{\n\t\tElementType:  elementType,\n\t\tContainsNull: containsNull,\n\t}\n}\n\nvar (\n\tBOOLEAN       = BooleanType{}\n\tBYTE          = ByteType{}\n\tSHORT         = ShortType{}\n\tINTEGER       = IntegerType{}\n\tLONG          = LongType{}\n\tFLOAT         = FloatType{}\n\tDOUBLE        = DoubleType{}\n\tDATE          = DateType{}\n\tTIMESTAMP     = TimestampType{}\n\tTIMESTAMP_NTZ = TimestampNtzType{}\n\tSTRING        = StringType{}\n)\n"
  },
  {
    "path": "spark/sql/types/datatype_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"testing\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestArrayType_TypeName(t *testing.T) {\n\ttests := []struct {\n\t\tname         string\n\t\tarrayType    ArrayType\n\t\texpectedName string\n\t}{\n\t\t{\n\t\t\tname: \"Array of integers\",\n\t\t\tarrayType: ArrayType{\n\t\t\t\tElementType:  INTEGER,\n\t\t\t\tContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Array<Integer>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Array of strings with nulls\",\n\t\t\tarrayType: ArrayType{\n\t\t\t\tElementType:  STRING,\n\t\t\t\tContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Array<String>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Nested array\",\n\t\t\tarrayType: ArrayType{\n\t\t\t\tElementType: ArrayType{\n\t\t\t\t\tElementType:  DOUBLE,\n\t\t\t\t\tContainsNull: false,\n\t\t\t\t},\n\t\t\t\tContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Array<Array<Double>>\",\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tassert.Equal(t, tt.expectedName, tt.arrayType.TypeName())\n\t\t})\n\t}\n}\n\nfunc TestArrayType_IsNumeric(t *testing.T) {\n\tarrayType := ArrayType{\n\t\tElementType:  INTEGER,\n\t\tContainsNull: false,\n\t}\n\tassert.False(t, arrayType.IsNumeric())\n}\n\nfunc TestArrayType_ToArrowType(t *testing.T) {\n\ttests := []struct {\n\t\tname      string\n\t\tarrayType ArrayType\n\t\tvalidate  func(t *testing.T, arrowType arrow.DataType)\n\t}{\n\t\t{\n\t\t\tname: \"Array of integers\",\n\t\t\tarrayType: ArrayType{\n\t\t\t\tElementType:  INTEGER,\n\t\t\t\tContainsNull: false,\n\t\t\t},\n\t\t\tvalidate: func(t *testing.T, arrowType arrow.DataType) {\n\t\t\t\tlistType, ok := arrowType.(*arrow.ListType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, arrow.PrimitiveTypes.Int32, listType.Elem())\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"Array of strings\",\n\t\t\tarrayType: ArrayType{\n\t\t\t\tElementType:  STRING,\n\t\t\t\tContainsNull: true,\n\t\t\t},\n\t\t\tvalidate: func(t *testing.T, arrowType arrow.DataType) {\n\t\t\t\tlistType, ok := arrowType.(*arrow.ListType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, arrow.BinaryTypes.String, listType.Elem())\n\t\t\t},\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tarrowType := tt.arrayType.ToArrowType()\n\t\t\ttt.validate(t, arrowType)\n\t\t})\n\t}\n}\n\nfunc TestMapType_TypeName(t *testing.T) {\n\ttests := []struct {\n\t\tname         string\n\t\tmapType      MapType\n\t\texpectedName string\n\t}{\n\t\t{\n\t\t\tname: \"Map of string to integer\",\n\t\t\tmapType: MapType{\n\t\t\t\tKeyType:           STRING,\n\t\t\t\tValueType:         INTEGER,\n\t\t\t\tValueContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Map<String,Integer>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Map of integer to array\",\n\t\t\tmapType: MapType{\n\t\t\t\tKeyType: INTEGER,\n\t\t\t\tValueType: ArrayType{\n\t\t\t\t\tElementType:  STRING,\n\t\t\t\t\tContainsNull: true,\n\t\t\t\t},\n\t\t\t\tValueContainsNull: true,\n\t\t\t},\n\t\t\texpectedName: \"Map<Integer,Array<String>>\",\n\t\t},\n\t\t{\n\t\t\tname: \"Nested map\",\n\t\t\tmapType: MapType{\n\t\t\t\tKeyType: STRING,\n\t\t\t\tValueType: MapType{\n\t\t\t\t\tKeyType:           STRING,\n\t\t\t\t\tValueType:         DOUBLE,\n\t\t\t\t\tValueContainsNull: false,\n\t\t\t\t},\n\t\t\t\tValueContainsNull: false,\n\t\t\t},\n\t\t\texpectedName: \"Map<String,Map<String,Double>>\",\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tassert.Equal(t, tt.expectedName, tt.mapType.TypeName())\n\t\t})\n\t}\n}\n\nfunc TestMapType_IsNumeric(t *testing.T) {\n\tmapType := MapType{\n\t\tKeyType:           STRING,\n\t\tValueType:         INTEGER,\n\t\tValueContainsNull: false,\n\t}\n\tassert.False(t, mapType.IsNumeric())\n}\n\nfunc TestMapType_ToArrowType(t *testing.T) {\n\ttests := []struct {\n\t\tname     string\n\t\tmapType  MapType\n\t\tvalidate func(t *testing.T, arrowType arrow.DataType)\n\t}{\n\t\t{\n\t\t\tname: \"Map of string to integer\",\n\t\t\tmapType: MapType{\n\t\t\t\tKeyType:           STRING,\n\t\t\t\tValueType:         INTEGER,\n\t\t\t\tValueContainsNull: true,\n\t\t\t},\n\t\t\tvalidate: func(t *testing.T, arrowType arrow.DataType) {\n\t\t\t\tmapType, ok := arrowType.(*arrow.MapType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, arrow.BinaryTypes.String, mapType.KeyType())\n\t\t\t\tassert.Equal(t, arrow.PrimitiveTypes.Int32, mapType.ItemType())\n\t\t\t},\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tarrowType := tt.mapType.ToArrowType()\n\t\t\ttt.validate(t, arrowType)\n\t\t})\n\t}\n}\n\nfunc TestMakeArrayType(t *testing.T) {\n\tarrayType := MakeArrayType(STRING, true)\n\n\tassert.Equal(t, STRING, arrayType.ElementType)\n\tassert.True(t, arrayType.ContainsNull)\n\tassert.Equal(t, \"Array<String>\", arrayType.TypeName())\n}\n\nfunc TestComplexTypeNesting(t *testing.T) {\n\t// Test complex nested structure: Array<Map<String, Array<Integer>>>\n\tinnerArray := ArrayType{\n\t\tElementType:  INTEGER,\n\t\tContainsNull: false,\n\t}\n\n\tmapType := MapType{\n\t\tKeyType:           STRING,\n\t\tValueType:         innerArray,\n\t\tValueContainsNull: true,\n\t}\n\n\touterArray := ArrayType{\n\t\tElementType:  mapType,\n\t\tContainsNull: false,\n\t}\n\n\texpectedName := \"Array<Map<String,Array<Integer>>>\"\n\tassert.Equal(t, expectedName, outerArray.TypeName())\n}\n"
  },
  {
    "path": "spark/sql/types/row.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"encoding/base64\"\n\t\"encoding/json\"\n\t\"fmt\"\n\t\"maps\"\n\t\"reflect\"\n\t\"time\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal128\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal256\"\n)\n\ntype Row interface {\n\t// At returns field's value at the given index within a [Row].\n\t// It returns nil for invalid indices.\n\tAt(index int) any\n\t// Value returns field's value of the given column's name within a [Row].\n\t// It returns nil for invalid column's name.\n\tValue(name string) any\n\t// Values returns values of all fields within a [Row] as a slice of any.\n\tValues() []any\n\t// Len returns the number of fields within a [Row].\n\tLen() int\n\tFieldNames() []string\n\t// ToJsonString converts the Row to a JSON string representation.\n\t// Returns an error if the row contains data that cannot be properly represented in JSON.\n\tToJsonString() (string, error)\n}\n\ntype rowImpl struct {\n\tvalues  []any\n\toffsets map[string]int\n}\n\nfunc (r *rowImpl) At(index int) any {\n\tif index < 0 || index > len(r.values) {\n\t\treturn nil\n\t}\n\treturn r.values[index]\n}\n\nfunc (r *rowImpl) Value(name string) any {\n\tidx, ok := r.offsets[name]\n\tif !ok {\n\t\treturn nil\n\t}\n\treturn r.values[idx]\n}\n\nfunc (r *rowImpl) Values() []any {\n\treturn r.values\n}\n\nfunc (r *rowImpl) Len() int {\n\treturn len(r.values)\n}\n\nfunc (r *rowImpl) FieldNames() []string {\n\tnames := make([]string, len(r.offsets))\n\t// Sort the field names to make the output deterministic.\n\tfor k, v := range maps.All(r.offsets) {\n\t\tnames[v] = k\n\t}\n\treturn names\n}\n\nfunc (r *rowImpl) ToJsonString() (string, error) {\n\tjsonMap := make(map[string]any)\n\tfieldNames := r.FieldNames()\n\n\tfor i, fieldName := range fieldNames {\n\t\tvalue := r.values[i]\n\t\tconvertedValue, err := convertToJsonValue(value)\n\t\tif err != nil {\n\t\t\treturn \"\", fmt.Errorf(\"failed to convert field '%s': %w\", fieldName, err)\n\t\t}\n\t\tjsonMap[fieldName] = convertedValue\n\t}\n\n\tjsonBytes, err := json.Marshal(jsonMap)\n\tif err != nil {\n\t\treturn \"\", fmt.Errorf(\"failed to marshal JSON: %w\", err)\n\t}\n\n\treturn string(jsonBytes), nil\n}\n\nfunc convertToJsonValue(value any) (any, error) {\n\tif value == nil {\n\t\treturn nil, nil\n\t}\n\n\tswitch v := value.(type) {\n\tcase bool, string, int8, int16, int32, int64, float32, float64:\n\t\treturn v, nil\n\n\tcase []byte:\n\t\treturn base64.StdEncoding.EncodeToString(v), nil\n\n\tcase decimal128.Num:\n\t\treturn v.BigInt().String(), nil\n\n\tcase decimal256.Num:\n\t\treturn v.BigInt().String(), nil\n\n\tcase arrow.Timestamp:\n\t\tepochUs := int64(v)\n\t\tt := time.Unix(epochUs/1000000, (epochUs%1000000)*1000).UTC()\n\t\treturn t.Format(time.RFC3339), nil\n\n\tcase arrow.Date32:\n\t\tepochDays := int64(v)\n\t\tepochTime := time.Date(1970, 1, 1, 0, 0, 0, 0, time.UTC).AddDate(0, 0, int(epochDays))\n\t\treturn epochTime.Format(\"2006-01-02\"), nil\n\n\tcase arrow.Date64:\n\t\tepochMs := int64(v)\n\t\tt := time.Unix(epochMs/1000, (epochMs%1000)*1000000).UTC()\n\t\treturn t.Format(\"2006-01-02\"), nil\n\n\tcase time.Time:\n\t\tif v.IsZero() {\n\t\t\treturn nil, nil\n\t\t}\n\t\treturn v.Format(time.RFC3339), nil\n\n\tcase []any:\n\t\tresult := make([]any, len(v))\n\t\tfor i, item := range v {\n\t\t\tconvertedItem, err := convertToJsonValue(item)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, fmt.Errorf(\"failed to convert array element at index %d: %w\", i, err)\n\t\t\t}\n\t\t\tresult[i] = convertedItem\n\t\t}\n\t\treturn result, nil\n\n\tcase map[any]any:\n\t\tresult := make(map[string]any)\n\t\tfor key, val := range v {\n\t\t\tkeyStr, ok := key.(string)\n\t\t\tif !ok {\n\t\t\t\treturn nil, fmt.Errorf(\"map key must be string for JSON conversion, got %T\", key)\n\t\t\t}\n\t\t\tconvertedVal, err := convertToJsonValue(val)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, fmt.Errorf(\"failed to convert map value for key '%s': %w\", keyStr, err)\n\t\t\t}\n\t\t\tresult[keyStr] = convertedVal\n\t\t}\n\t\treturn result, nil\n\n\tcase map[string]any:\n\t\tresult := make(map[string]any)\n\t\tfor key, val := range v {\n\t\t\tconvertedVal, err := convertToJsonValue(val)\n\t\t\tif err != nil {\n\t\t\t\treturn nil, fmt.Errorf(\"failed to convert map value for key '%s': %w\", key, err)\n\t\t\t}\n\t\t\tresult[key] = convertedVal\n\t\t}\n\t\treturn result, nil\n\n\tdefault:\n\t\t// Use reflection to handle custom types that have basic types as their underlying type.\n\t\t// For example, a custom type like \"type MyInt int32\" would not match the explicit\n\t\t// int32 case above, but would match reflect.Int32 here. This ensures we can still\n\t\t// convert custom integer, float, bool, and string types to their JSON representations.\n\t\trv := reflect.ValueOf(value)\n\t\tswitch rv.Kind() {\n\t\tcase reflect.Slice, reflect.Array:\n\t\t\tlength := rv.Len()\n\t\t\tresult := make([]any, length)\n\t\t\tfor i := 0; i < length; i++ {\n\t\t\t\tconvertedItem, err := convertToJsonValue(rv.Index(i).Interface())\n\t\t\t\tif err != nil {\n\t\t\t\t\treturn nil, fmt.Errorf(\"failed to convert array element at index %d: %w\", i, err)\n\t\t\t\t}\n\t\t\t\tresult[i] = convertedItem\n\t\t\t}\n\t\t\treturn result, nil\n\n\t\tcase reflect.Map:\n\t\t\tif rv.Type().Key().Kind() != reflect.String {\n\t\t\t\treturn nil, fmt.Errorf(\"map key must be string for JSON conversion, got %s\", rv.Type().Key().Kind())\n\t\t\t}\n\t\t\tresult := make(map[string]any)\n\t\t\tfor _, key := range rv.MapKeys() {\n\t\t\t\tkeyStr := key.String()\n\t\t\t\tval := rv.MapIndex(key)\n\t\t\t\tconvertedVal, err := convertToJsonValue(val.Interface())\n\t\t\t\tif err != nil {\n\t\t\t\t\treturn nil, fmt.Errorf(\"failed to convert map value for key '%s': %w\", keyStr, err)\n\t\t\t\t}\n\t\t\t\tresult[keyStr] = convertedVal\n\t\t\t}\n\t\t\treturn result, nil\n\n\t\tcase reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64:\n\t\t\treturn rv.Int(), nil\n\n\t\tcase reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64:\n\t\t\treturn rv.Uint(), nil\n\n\t\tcase reflect.Float32, reflect.Float64:\n\t\t\treturn rv.Float(), nil\n\n\t\tcase reflect.Bool:\n\t\t\treturn rv.Bool(), nil\n\n\t\tcase reflect.String:\n\t\t\treturn rv.String(), nil\n\n\t\tdefault:\n\t\t\treturn fmt.Sprintf(\"%v\", value), nil\n\t\t}\n\t}\n}\n"
  },
  {
    "path": "spark/sql/types/row_json_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"encoding/json\"\n\t\"testing\"\n\t\"time\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal128\"\n\t\"github.com/apache/arrow-go/v18/arrow/decimal256\"\n\t\"github.com/stretchr/testify/assert\"\n\t\"github.com/stretchr/testify/require\"\n)\n\nfunc TestRowToJsonString(t *testing.T) {\n\ttests := []struct {\n\t\tname     string\n\t\trow      Row\n\t\texpected string\n\t\thasError bool\n\t}{\n\t\t{\n\t\t\tname: \"basic types\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\t\"hello\",\n\t\t\t\t\tint32(42),\n\t\t\t\t\tint64(123),\n\t\t\t\t\tfloat64(3.14),\n\t\t\t\t\ttrue,\n\t\t\t\t\tnil,\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"str_col\":    0,\n\t\t\t\t\t\"int_col\":    1,\n\t\t\t\t\t\"long_col\":   2,\n\t\t\t\t\t\"double_col\": 3,\n\t\t\t\t\t\"bool_col\":   4,\n\t\t\t\t\t\"null_col\":   5,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"bool_col\":true,\"double_col\":3.14,\"int_col\":42,\"long_col\":123,\"null_col\":null,\"str_col\":\"hello\"}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"binary data\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\t[]byte(\"hello world\"),\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"binary_col\": 0,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"binary_col\":\"aGVsbG8gd29ybGQ=\"}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"decimal types\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\tdecimal128.FromI64(123456),\n\t\t\t\t\tdecimal256.FromI64(789012),\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"decimal128_col\": 0,\n\t\t\t\t\t\"decimal256_col\": 1,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"decimal128_col\":\"123456\",\"decimal256_col\":\"789012\"}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"timestamp and date\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\tarrow.Timestamp(1686981953115000), // microseconds\n\t\t\t\t\tarrow.Date32(19521),               // days since epoch (2023-06-13)\n\t\t\t\t\tarrow.Date64(1686981953115),       // milliseconds\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"timestamp_col\": 0,\n\t\t\t\t\t\"date32_col\":    1,\n\t\t\t\t\t\"date64_col\":    2,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"date32_col\":\"2023-06-13\",\"date64_col\":\"2023-06-17\",\"timestamp_col\":\"2023-06-17T06:05:53Z\"}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"arrays\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\t[]any{1, 2, 3},\n\t\t\t\t\t[]any{\"a\", \"b\", \"c\"},\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"int_array\": 0,\n\t\t\t\t\t\"str_array\": 1,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"int_array\":[1,2,3],\"str_array\":[\"a\",\"b\",\"c\"]}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"valid string map\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\tmap[string]any{\n\t\t\t\t\t\t\"key1\": \"value1\",\n\t\t\t\t\t\t\"key2\": 42,\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"map_col\": 0,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"map_col\":{\"key1\":\"value1\",\"key2\":42}}`,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname: \"invalid map with non-string keys\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\tmap[any]any{\n\t\t\t\t\t\t42:     \"value1\",\n\t\t\t\t\t\t\"key2\": \"value2\",\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"map_col\": 0,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: \"\",\n\t\t\thasError: true,\n\t\t},\n\t\t{\n\t\t\tname: \"nested structures\",\n\t\t\trow: &rowImpl{\n\t\t\t\tvalues: []any{\n\t\t\t\t\t[]any{\n\t\t\t\t\t\tmap[string]any{\n\t\t\t\t\t\t\t\"nested_key\": \"nested_value\",\n\t\t\t\t\t\t\t\"nested_num\": 123,\n\t\t\t\t\t\t},\n\t\t\t\t\t},\n\t\t\t\t},\n\t\t\t\toffsets: map[string]int{\n\t\t\t\t\t\"nested_col\": 0,\n\t\t\t\t},\n\t\t\t},\n\t\t\texpected: `{\"nested_col\":[{\"nested_key\":\"nested_value\",\"nested_num\":123}]}`,\n\t\t\thasError: false,\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tresult, err := tt.row.ToJsonString()\n\n\t\t\tif tt.hasError {\n\t\t\t\tassert.Error(t, err)\n\t\t\t\tassert.Empty(t, result)\n\t\t\t} else {\n\t\t\t\trequire.NoError(t, err)\n\n\t\t\t\t// Verify the result is valid JSON\n\t\t\t\tvar parsed map[string]any\n\t\t\t\terr = json.Unmarshal([]byte(result), &parsed)\n\t\t\t\trequire.NoError(t, err)\n\n\t\t\t\t// Verify the expected content (comparing parsed JSON to avoid key ordering issues)\n\t\t\t\tvar expected map[string]any\n\t\t\t\terr = json.Unmarshal([]byte(tt.expected), &expected)\n\t\t\t\trequire.NoError(t, err)\n\n\t\t\t\tassert.Equal(t, expected, parsed)\n\t\t\t}\n\t\t})\n\t}\n}\n\nfunc TestConvertToJsonValue(t *testing.T) {\n\ttests := []struct {\n\t\tname     string\n\t\tinput    any\n\t\texpected any\n\t\thasError bool\n\t}{\n\t\t{\n\t\t\tname:     \"nil value\",\n\t\t\tinput:    nil,\n\t\t\texpected: nil,\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname:     \"time.Time\",\n\t\t\tinput:    time.Date(2023, 6, 17, 10, 5, 53, 0, time.UTC),\n\t\t\texpected: \"2023-06-17T10:05:53Z\",\n\t\t\thasError: false,\n\t\t},\n\t\t{\n\t\t\tname:     \"zero time.Time\",\n\t\t\tinput:    time.Time{},\n\t\t\texpected: nil,\n\t\t\thasError: false,\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tresult, err := convertToJsonValue(tt.input)\n\n\t\t\tif tt.hasError {\n\t\t\t\tassert.Error(t, err)\n\t\t\t} else {\n\t\t\t\trequire.NoError(t, err)\n\t\t\t\tassert.Equal(t, tt.expected, result)\n\t\t\t}\n\t\t})\n\t}\n}\n"
  },
  {
    "path": "spark/sql/types/row_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"testing\"\n\n\t\"github.com/stretchr/testify/require\"\n)\n\n// rowImplTest is a read-only sample [Row] to be used in all tests.\nvar rowImplSample rowImpl = rowImpl{\n\tvalues: []any{1, 2, 3, 4, 5},\n\toffsets: map[string]int{\n\t\t\"five\":  4,\n\t\t\"one\":   0,\n\t\t\"two\":   1,\n\t\t\"four\":  3,\n\t\t\"three\": 2,\n\t},\n}\n\nfunc TestRowImpl_At(t *testing.T) {\n\ttestCases := []struct {\n\t\tname  string\n\t\tinput int\n\t\texp   any\n\t}{\n\t\t{\n\t\t\tname:  \"index within range\",\n\t\t\tinput: 2,\n\t\t\texp:   3,\n\t\t},\n\t\t{\n\t\t\tname:  \"index out of range\",\n\t\t\tinput: 6,\n\t\t\texp:   nil,\n\t\t},\n\t\t{\n\t\t\tname:  \"negative index\",\n\t\t\tinput: -1,\n\t\t\texp:   nil,\n\t\t},\n\t}\n\n\tfor _, tc := range testCases {\n\t\tt.Run(tc.name, func(t *testing.T) {\n\t\t\tact := rowImplSample.At(tc.input)\n\t\t\trequire.Equal(t, tc.exp, act)\n\t\t})\n\t}\n}\n\nfunc TestRowImpl_Value(t *testing.T) {\n\ttestCases := []struct {\n\t\tname  string\n\t\tinput string\n\t\texp   any\n\t}{\n\t\t{\n\t\t\tname:  \"valid field name\",\n\t\t\tinput: \"two\",\n\t\t\texp:   2,\n\t\t},\n\t\t{\n\t\t\tname:  \"invalid field name\",\n\t\t\tinput: \"six\",\n\t\t\texp:   nil,\n\t\t},\n\t}\n\n\tfor _, tc := range testCases {\n\t\tt.Run(tc.name, func(t *testing.T) {\n\t\t\tact := rowImplSample.Value(tc.input)\n\t\t\trequire.Equal(t, tc.exp, act)\n\t\t})\n\t}\n}\n\nfunc TestRowImpl_Values(t *testing.T) {\n\texp := []any{1, 2, 3, 4, 5}\n\tact := rowImplSample.Values()\n\trequire.Equal(t, exp, act)\n}\n\nfunc TestRowImpl_Len(t *testing.T) {\n\texp := 5\n\tact := rowImplSample.Len()\n\trequire.Equal(t, exp, act)\n}\n\nfunc TestRowImpl_FieldNames(t *testing.T) {\n\texp := []string{\"one\", \"two\", \"three\", \"four\", \"five\"}\n\tact := rowImplSample.FieldNames()\n\trequire.ElementsMatch(t, exp, act)\n}\n"
  },
  {
    "path": "spark/sql/types/structtype.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"fmt\"\n\t\"strings\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n)\n\n// StructField represents a field in a StructType.\ntype StructField struct {\n\tName     string\n\tDataType DataType\n\tNullable bool // default should be true\n\tMetadata *string\n}\n\nfunc (t *StructField) ToArrowType() arrow.Field {\n\treturn arrow.Field{\n\t\tName:     t.Name,\n\t\tType:     t.DataType.ToArrowType(),\n\t\tNullable: t.Nullable,\n\t}\n}\n\nfunc (t *StructField) buildFormattedString(prefix string, target *string) {\n\tif target == nil {\n\t\treturn\n\t}\n\n\tswitch t.DataType.(type) {\n\tcase ArrayType:\n\t\t*target += fmt.Sprintf(\"%s-- %s: array (nullable = %t)\\n\",\n\t\t\tprefix, t.Name, t.Nullable)\n\t\t*target += fmt.Sprintf(\"%s    |-- element: %s (valueContainsNull = %t)\\n\",\n\t\t\tprefix, t.DataType.(ArrayType).ElementType.TypeName(), t.Nullable)\n\tcase MapType:\n\t\t*target += fmt.Sprintf(\"%s-- %s: map (nullable = %t)\\n\",\n\t\t\tprefix, t.Name, t.Nullable)\n\t\t*target += fmt.Sprintf(\"%s    |-- key: %s\\n\",\n\t\t\tprefix, t.DataType.(MapType).KeyType.TypeName())\n\t\t*target += fmt.Sprintf(\"%s    |-- value: %s (valueContainsNull = %t)\\n\",\n\t\t\tprefix, t.DataType.(MapType).ValueType.TypeName(), t.Nullable)\n\tcase StructType:\n\t\t*target += fmt.Sprintf(\"%s-- %s: structtype (nullable = %t)\\n\",\n\t\t\tprefix, t.Name, t.Nullable)\n\t\tfor _, field := range t.DataType.(StructType).Fields {\n\t\t\tfield.buildFormattedString(prefix+\"    |\", target)\n\t\t}\n\tdefault:\n\t\t*target += fmt.Sprintf(\"%s-- %s: %s (nullable = %t)\\n\", prefix, t.Name,\n\t\t\tstrings.ToLower(t.DataType.TypeName()), t.Nullable)\n\t}\n}\n\n// StructType represents a struct type.\ntype StructType struct {\n\tFields []StructField\n}\n\nfunc (t StructType) TypeName() string {\n\treturn \"structtype\"\n}\n\nfunc (t StructType) IsNumeric() bool {\n\treturn false\n}\n\nfunc (t StructType) ToArrowType() arrow.DataType {\n\tfields := make([]arrow.Field, len(t.Fields))\n\tfor i, f := range t.Fields {\n\t\tfields[i] = f.ToArrowType()\n\t}\n\treturn arrow.StructOf(fields...)\n}\n\nfunc (t *StructType) TreeString() string {\n\ttree := string(\"root\\n\")\n\tprefix := \" |\"\n\tfor _, f := range t.Fields {\n\t\tf.buildFormattedString(prefix, &tree)\n\t}\n\treturn tree + \"\\n\"\n}\n\nfunc StructOf(fields ...StructField) *StructType {\n\treturn &StructType{Fields: fields}\n}\n\nfunc NewStructField(name string, dataType DataType) StructField {\n\treturn StructField{\n\t\tName:     name,\n\t\tDataType: dataType,\n\t\tNullable: true,\n\t}\n}\n"
  },
  {
    "path": "spark/sql/types/structtype_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage types\n\nimport (\n\t\"strings\"\n\t\"testing\"\n\n\t\"github.com/apache/arrow-go/v18/arrow\"\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestStructOf(t *testing.T) {\n\ts := StructOf(NewStructField(\"col1\", BYTE))\n\tassert.Len(t, s.Fields, 1)\n}\n\nfunc TestStructType_TypeName(t *testing.T) {\n\tstructType := StructType{\n\t\tFields: []StructField{\n\t\t\t{Name: \"field1\", DataType: INTEGER},\n\t\t\t{Name: \"field2\", DataType: STRING},\n\t\t},\n\t}\n\tassert.Equal(t, \"structtype\", structType.TypeName())\n}\n\nfunc TestStructType_IsNumeric(t *testing.T) {\n\tstructType := StructType{\n\t\tFields: []StructField{\n\t\t\t{Name: \"field1\", DataType: INTEGER},\n\t\t},\n\t}\n\tassert.False(t, structType.IsNumeric())\n}\n\nfunc TestStructType_ToArrowType(t *testing.T) {\n\ttests := []struct {\n\t\tname       string\n\t\tstructType StructType\n\t\tvalidate   func(t *testing.T, arrowType arrow.DataType)\n\t}{\n\t\t{\n\t\t\tname: \"Simple struct with integer and string fields\",\n\t\t\tstructType: StructType{\n\t\t\t\tFields: []StructField{\n\t\t\t\t\t{Name: \"id\", DataType: INTEGER, Nullable: false},\n\t\t\t\t\t{Name: \"name\", DataType: STRING, Nullable: true},\n\t\t\t\t},\n\t\t\t},\n\t\t\tvalidate: func(t *testing.T, arrowType arrow.DataType) {\n\t\t\t\tstructType, ok := arrowType.(*arrow.StructType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, 2, structType.NumFields())\n\n\t\t\t\tfield1 := structType.Field(0)\n\t\t\t\tassert.Equal(t, \"id\", field1.Name)\n\t\t\t\tassert.Equal(t, arrow.PrimitiveTypes.Int32, field1.Type)\n\t\t\t\tassert.False(t, field1.Nullable)\n\n\t\t\t\tfield2 := structType.Field(1)\n\t\t\t\tassert.Equal(t, \"name\", field2.Name)\n\t\t\t\tassert.Equal(t, arrow.BinaryTypes.String, field2.Type)\n\t\t\t\tassert.True(t, field2.Nullable)\n\t\t\t},\n\t\t},\n\t\t{\n\t\t\tname: \"Struct with array field\",\n\t\t\tstructType: StructType{\n\t\t\t\tFields: []StructField{\n\t\t\t\t\t{Name: \"items\", DataType: ArrayType{ElementType: STRING, ContainsNull: true}, Nullable: false},\n\t\t\t\t},\n\t\t\t},\n\t\t\tvalidate: func(t *testing.T, arrowType arrow.DataType) {\n\t\t\t\tstructType, ok := arrowType.(*arrow.StructType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, 1, structType.NumFields())\n\n\t\t\t\tfield := structType.Field(0)\n\t\t\t\tassert.Equal(t, \"items\", field.Name)\n\t\t\t\tlistType, ok := field.Type.(*arrow.ListType)\n\t\t\t\tassert.True(t, ok)\n\t\t\t\tassert.Equal(t, arrow.BinaryTypes.String, listType.Elem())\n\t\t\t},\n\t\t},\n\t}\n\n\tfor _, tt := range tests {\n\t\tt.Run(tt.name, func(t *testing.T) {\n\t\t\tarrowType := tt.structType.ToArrowType()\n\t\t\ttt.validate(t, arrowType)\n\t\t})\n\t}\n}\n\nfunc TestStructType_ToArrowType_ReturnType(t *testing.T) {\n\t// Test that ToArrowType returns arrow.DataType interface, not *arrow.StructType\n\tstructType := StructType{\n\t\tFields: []StructField{\n\t\t\t{Name: \"field1\", DataType: INTEGER},\n\t\t},\n\t}\n\n\tarrowType := structType.ToArrowType()\n\n\t// This should compile and work correctly with the interface\n\tdataType := arrowType\n\tassert.NotNil(t, dataType)\n\n\t// But we should still be able to cast it to the concrete type\n\tconcreteType, ok := arrowType.(*arrow.StructType)\n\tassert.True(t, ok)\n\tassert.Equal(t, 1, concreteType.NumFields())\n}\n\nfunc TestTreeString(t *testing.T) {\n\tc := NewStructField(\"col1\", STRING)\n\tc.Nullable = false\n\ts := StructOf(\n\t\tc,\n\t\tNewStructField(\"col2\", INTEGER),\n\t\tNewStructField(\"col3\", DATE),\n\t)\n\tassert.Len(t, s.Fields, 3)\n\tts := s.TreeString()\n\tassert.Contains(t, ts, \"|-- col1: string (nullable = false\")\n\tassert.Contains(t, ts, \"|-- col2: integer (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- col3: date (nullable = true)\")\n}\n\nfunc TestTreeString_ComplexNestedTypes(t *testing.T) {\n\t// Create a complex nested structure with maps, arrays, and nested structs\n\tnestedStruct := StructOf(\n\t\tNewStructField(\"nested_id\", INTEGER),\n\t\tNewStructField(\"nested_name\", STRING),\n\t)\n\n\tarrayOfStrings := ArrayType{\n\t\tElementType:  STRING,\n\t\tContainsNull: true,\n\t}\n\n\tmapOfIntToString := MapType{\n\t\tKeyType:           INTEGER,\n\t\tValueType:         STRING,\n\t\tValueContainsNull: true,\n\t}\n\n\tarrayOfMaps := ArrayType{\n\t\tElementType: MapType{\n\t\t\tKeyType:           STRING,\n\t\t\tValueType:         DOUBLE,\n\t\t\tValueContainsNull: false,\n\t\t},\n\t\tContainsNull: true,\n\t}\n\n\tcomplexStruct := StructOf(\n\t\tNewStructField(\"id\", INTEGER),\n\t\tNewStructField(\"name\", STRING),\n\t\tNewStructField(\"tags\", arrayOfStrings),\n\t\tNewStructField(\"metadata\", mapOfIntToString),\n\t\tNewStructField(\"scores\", arrayOfMaps),\n\t\tNewStructField(\"profile\", *nestedStruct),\n\t\tNewStructField(\"active\", BOOLEAN),\n\t)\n\n\tts := complexStruct.TreeString()\n\n\t// Verify the tree string contains all expected elements\n\tassert.Contains(t, ts, \"root\")\n\tassert.Contains(t, ts, \"|-- id: integer (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- name: string (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- tags: array (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- metadata: map (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- scores: array (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- profile: structtype (nullable = true)\")\n\tassert.Contains(t, ts, \"|-- active: boolean (nullable = true)\")\n\n\t// Verify the structure starts with \"root\" and ends with newlines\n\tassert.True(t, strings.HasPrefix(ts, \"root\\n\"))\n\tassert.True(t, strings.HasSuffix(ts, \"\\n\"))\n\n\t// Verify the correct number of field lines (excluding root and trailing newline)\n\tlines := strings.Split(strings.TrimSpace(ts), \"\\n\")\n\tassert.Equal(t, len(complexStruct.Fields)+7, len(lines)) // root + number of fields\n\n\t// Verify the prefix format for all field lines\n\tfor i := 1; i < len(lines); i++ {\n\t\tassert.True(t, strings.HasPrefix(lines[i], \" |\"))\n\t}\n}\n"
  },
  {
    "path": "spark/sql/utils/check.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage utils\n\nfunc WarnOnError(f func() error, h func(e error)) {\n\tif err := f(); err != nil {\n\t\th(err)\n\t}\n}\n"
  },
  {
    "path": "spark/sql/utils/check_test.go",
    "content": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage utils\n\nimport (\n\t\"testing\"\n\n\t\"github.com/stretchr/testify/assert\"\n)\n\nfunc TestWarnOnError(t *testing.T) {\n\tWarnOnError(func() error {\n\t\treturn nil\n\t}, func(e error) {\n\t\tt.Errorf(\"Unexpected error: %v\", e)\n\t})\n\n\tcalled := 0\n\tWarnOnError(func() error {\n\t\treturn assert.AnError\n\t}, func(e error) {\n\t\tcalled++\n\t\tassert.Equal(t, assert.AnError, e)\n\t})\n\tassert.Equalf(t, 1, called, \"Expected error handler to be called once, got %v\", called)\n}\n"
  },
  {
    "path": "spark/sql/utils/consts.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage utils\n\nimport proto \"github.com/apache/spark-connect-go/internal/generated\"\n\ntype ExplainMode int\n\nconst (\n\tExplainModeSimple    ExplainMode = iota\n\tExplainModeExtended  ExplainMode = iota\n\tExplainModeCodegen   ExplainMode = iota\n\tExplainModeCost      ExplainMode = iota\n\tExplainModeFormatted ExplainMode = iota\n)\n\ntype StorageLevel int\n\nconst (\n\tStorageLevelDiskOnly          StorageLevel = iota\n\tStorageLevelDiskOnly2         StorageLevel = iota\n\tStorageLevelDiskOnly3         StorageLevel = iota\n\tStorageLevelMemoryAndDisk     StorageLevel = iota\n\tStorageLevelMemoryAndDisk2    StorageLevel = iota\n\tStorageLevelMemoryOnly        StorageLevel = iota\n\tStorageLevelMemoryOnly2       StorageLevel = iota\n\tStorageLevelMemoyAndDiskDeser StorageLevel = iota\n\tStorageLevelNone              StorageLevel = iota\n\tStorageLevelOffHeap           StorageLevel = iota\n)\n\nfunc ToProtoStorageLevel(level StorageLevel) *proto.StorageLevel {\n\tswitch level {\n\tcase StorageLevelDiskOnly:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: false, Replication: 1}\n\tcase StorageLevelDiskOnly2:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: false, Replication: 2}\n\tcase StorageLevelDiskOnly3:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: false, Replication: 3}\n\tcase StorageLevelMemoryAndDisk:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: true, Replication: 1}\n\tcase StorageLevelMemoryAndDisk2:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: true, Replication: 2}\n\tcase StorageLevelMemoryOnly:\n\t\treturn &proto.StorageLevel{UseDisk: false, UseMemory: true, Replication: 1}\n\tcase StorageLevelMemoryOnly2:\n\t\treturn &proto.StorageLevel{UseDisk: false, UseMemory: true, Replication: 2}\n\tcase StorageLevelMemoyAndDiskDeser:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: true, Replication: 1, Deserialized: true}\n\tcase StorageLevelOffHeap:\n\t\treturn &proto.StorageLevel{UseDisk: true, UseMemory: true, UseOffHeap: true, Replication: 1}\n\tdefault:\n\t\treturn &proto.StorageLevel{UseDisk: false, UseMemory: false, UseOffHeap: false, Replication: 1}\n\t}\n}\n\nfunc FromProtoStorageLevel(level *proto.StorageLevel) StorageLevel {\n\tif level.UseDisk && level.UseMemory && level.Replication <= 1 && !level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelMemoryAndDisk\n\t} else if level.UseDisk && level.UseMemory && level.Replication == 2 && !level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelMemoryAndDisk2\n\t} else if level.UseDisk && !level.UseMemory && level.Replication == 3 &&\n\t\t!level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelDiskOnly3\n\t} else if level.UseDisk && !level.UseMemory && level.Replication == 2 &&\n\t\t!level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelDiskOnly2\n\t} else if level.UseDisk && !level.UseMemory && level.Replication <= 1 &&\n\t\t!level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelDiskOnly\n\t} else if !level.UseDisk && level.UseMemory && level.Replication <= 1 &&\n\t\t!level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelMemoryOnly\n\t} else if !level.UseDisk && level.UseMemory && level.Replication == 2 &&\n\t\t!level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelMemoryOnly2\n\t} else if level.UseDisk && level.UseMemory && level.Replication <= 1 && level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelMemoyAndDiskDeser\n\t} else if !level.UseDisk && !level.UseMemory && !level.Deserialized && !level.UseOffHeap {\n\t\treturn StorageLevelNone\n\t} else if level.UseOffHeap && !level.Deserialized {\n\t\treturn StorageLevelOffHeap\n\t}\n\treturn StorageLevelNone\n}\n\ntype JoinType int\n\nconst (\n\tJoinTypeInner      JoinType = iota\n\tJoinTypeLeftOuter  JoinType = iota\n\tJoinTypeRightOuter JoinType = iota\n\tJoinTypeFullOuter  JoinType = iota\n\tJoinTypeLeftSemi   JoinType = iota\n\tJoinTypeLeftAnti   JoinType = iota\n\tJoinTypeCross      JoinType = iota\n)\n\nfunc ToProtoJoinType(joinType JoinType) proto.Join_JoinType {\n\tswitch joinType {\n\tcase JoinTypeInner:\n\t\treturn proto.Join_JOIN_TYPE_INNER\n\tcase JoinTypeLeftOuter:\n\t\treturn proto.Join_JOIN_TYPE_LEFT_OUTER\n\tcase JoinTypeRightOuter:\n\t\treturn proto.Join_JOIN_TYPE_RIGHT_OUTER\n\tcase JoinTypeFullOuter:\n\t\treturn proto.Join_JOIN_TYPE_FULL_OUTER\n\tcase JoinTypeLeftSemi:\n\t\treturn proto.Join_JOIN_TYPE_LEFT_SEMI\n\tcase JoinTypeLeftAnti:\n\t\treturn proto.Join_JOIN_TYPE_LEFT_ANTI\n\tcase JoinTypeCross:\n\t\treturn proto.Join_JOIN_TYPE_CROSS\n\tdefault:\n\t\treturn proto.Join_JOIN_TYPE_INNER\n\t}\n}\n"
  },
  {
    "path": "spark/sql/utils/consts_test.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//\thttp://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage utils\n\nimport \"testing\"\n\nfunc TestStorageLevelConversion(t *testing.T) {\n\t// Given a list of all storage levels, convert them to and from proto and\n\t// check with the original value:\n\tfor _, level := range []StorageLevel{\n\t\tStorageLevelDiskOnly,\n\t\tStorageLevelDiskOnly2,\n\t\tStorageLevelDiskOnly3,\n\t\tStorageLevelMemoryAndDisk,\n\t\tStorageLevelMemoryAndDisk2,\n\t\tStorageLevelMemoryOnly,\n\t\tStorageLevelMemoryOnly2,\n\t\tStorageLevelMemoyAndDiskDeser,\n\t\tStorageLevelNone,\n\t\tStorageLevelOffHeap,\n\t} {\n\t\tprotoLevel := ToProtoStorageLevel(level)\n\t\tconvertedLevel := FromProtoStorageLevel(protoLevel)\n\t\tif level != convertedLevel {\n\t\t\tt.Errorf(\"Expected %v, got %v\", level, convertedLevel)\n\t\t}\n\t}\n}\n"
  },
  {
    "path": "spark/version.go",
    "content": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements.  See the NOTICE file distributed with\n// this work for additional information regarding copyright ownership.\n// The ASF licenses this file to You under the Apache License, Version 2.0\n// (the \"License\"); you may not use this file except in compliance with\n// the License.  You may obtain a copy of the License at\n//\n//    http://www.apache.org/licenses/LICENSE-2.0\n//\n// Unless required by applicable law or agreed to in writing, software\n// distributed under the License is distributed on an \"AS IS\" BASIS,\n// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n// See the License for the specific language governing permissions and\n// limitations under the License.\n\npackage spark\n\nfunc Version() string {\n\treturn \"4.0.0\"\n}\n"
  }
]