Showing preview only (2,184K chars total). Download the full file or copy to clipboard to get everything.
Repository: apache/spark-connect-go
Branch: master
Commit: 0e3d565b63e6
Files: 104
Total size: 2.1 MB
Directory structure:
gitextract_ljoq267_/
├── .asf.yaml
├── .github/
│ ├── PULL_REQUEST_TEMPLATE
│ ├── dependabot.yml
│ └── workflows/
│ └── build.yml
├── .gitignore
├── .gitmodules
├── .golangci.yml
├── CONTRIBUTING.md
├── LICENSE
├── Makefile
├── NOTICE
├── README.md
├── buf.gen.yaml
├── buf.work.yaml
├── cmd/
│ ├── spark-connect-example-raw-grpc-client/
│ │ └── main.go
│ └── spark-connect-example-spark-session/
│ └── main.go
├── dev/
│ ├── .rat-excludes
│ ├── README.md
│ ├── check-license
│ ├── gen.py
│ ├── release.py
│ └── requirements.txt
├── go.mod
├── go.sum
├── internal/
│ ├── generated/
│ │ ├── base.pb.go
│ │ ├── base_grpc.pb.go
│ │ ├── catalog.pb.go
│ │ ├── commands.pb.go
│ │ ├── common.pb.go
│ │ ├── example_plugins.pb.go
│ │ ├── expressions.pb.go
│ │ ├── ml.pb.go
│ │ ├── ml_common.pb.go
│ │ ├── pipelines.pb.go
│ │ ├── relations.pb.go
│ │ └── types.pb.go
│ └── tests/
│ └── integration/
│ ├── dataframe_test.go
│ ├── functions_test.go
│ ├── helper.go
│ ├── spark_runner.go
│ └── sql_test.go
├── java/
│ ├── .gitignore
│ ├── README.md
│ ├── build.sbt
│ ├── run.sh
│ └── src/
│ └── main/
│ └── scala/
│ └── org/
│ └── apache/
│ └── spark/
│ └── golang/
│ └── Runner.scala
├── merge_connect_go_pr.py
├── quick-start.md
└── spark/
├── client/
│ ├── base/
│ │ └── base.go
│ ├── channel/
│ │ ├── channel.go
│ │ ├── channel_test.go
│ │ └── compat.go
│ ├── client.go
│ ├── client_test.go
│ ├── conf.go
│ ├── options/
│ │ └── options.go
│ ├── retry.go
│ ├── retry_test.go
│ └── testutils/
│ └── utils.go
├── mocks/
│ ├── mock_executor.go
│ └── mocks.go
├── sparkerrors/
│ ├── errors.go
│ └── errors_test.go
├── sql/
│ ├── column/
│ │ ├── column.go
│ │ ├── column_test.go
│ │ ├── expressions.go
│ │ └── expressions_test.go
│ ├── dataframe.go
│ ├── dataframe_test.go
│ ├── dataframenafunctions.go
│ ├── dataframereader.go
│ ├── dataframereader_test.go
│ ├── dataframestatfunctions.go
│ ├── dataframewriter.go
│ ├── dataframewriter_test.go
│ ├── executeplanclient.go
│ ├── functions/
│ │ ├── buiitins.go
│ │ └── generated.go
│ ├── group.go
│ ├── group_test.go
│ ├── mocks_test.go
│ ├── plan.go
│ ├── plan_test.go
│ ├── sparksession.go
│ ├── sparksession_integration_test.go
│ ├── sparksession_test.go
│ ├── types/
│ │ ├── arrow.go
│ │ ├── arrow_test.go
│ │ ├── builtin.go
│ │ ├── builtin_test.go
│ │ ├── conversion.go
│ │ ├── conversion_test.go
│ │ ├── datatype.go
│ │ ├── datatype_test.go
│ │ ├── row.go
│ │ ├── row_json_test.go
│ │ ├── row_test.go
│ │ ├── structtype.go
│ │ └── structtype_test.go
│ └── utils/
│ ├── check.go
│ ├── check_test.go
│ ├── consts.go
│ └── consts_test.go
└── version.go
================================================
FILE CONTENTS
================================================
================================================
FILE: .asf.yaml
================================================
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features
---
github:
description: "Apache Spark Connect Client for Golang"
homepage: https://spark.apache.org/
enabled_merge_buttons:
merge: false
squash: true
rebase: true
features:
# Enable the "Issues" tab
issues: true
# Enable the "Projects" tab
projects: true
notifications:
pullrequests: reviews@spark.apache.org
issues: reviews@spark.apache.org
commits: commits@spark.apache.org
================================================
FILE: .github/PULL_REQUEST_TEMPLATE
================================================
<!--
Thanks for sending a pull request! Here are some tips for you:
1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
4. Be sure to keep the PR description updated to reflect all changes.
5. Please write your PR title to summarize what this PR proposes.
6. If possible, provide a concise example to reproduce the issue for a faster review.
7. If you want to add a new configuration, please read the guideline first for naming configurations in
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
8. If you want to add or modify an error type or message, please read the guideline first in
'core/src/main/resources/error/README.md'.
-->
### What changes were proposed in this pull request?
<!--
Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
2. If you fix some SQL features, you can provide some references of other DBMSes.
3. If there is design documentation, please add the link.
4. If there is a discussion in the mailing list, please add the link.
-->
### Why are the changes needed?
<!--
Please clarify why the changes are needed. For instance,
1. If you propose a new API, clarify the use case for a new API.
2. If you fix a bug, you can clarify why it is a bug.
-->
### Does this PR introduce _any_ user-facing change?
<!--
Note that it means *any* user-facing change including all aspects such as the documentation fix.
If 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.
If 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.
If no, write 'No'.
-->
### How was this patch tested?
<!--
If 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.
If 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.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
If 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.
-->
================================================
FILE: .github/dependabot.yml
================================================
# To get started with Dependabot version updates, you'll need to specify which
# package ecosystems to update and where the package manifests are located.
# Please see the documentation for all configuration options:
# https://docs.github.com/code-security/dependabot/dependabot-version-updates/configuration-options-for-the-dependabot.yml-file
version: 2
updates:
- package-ecosystem: "gomod" # See documentation for possible values
directory: "/" # Location of package manifests
schedule:
interval: "weekly"
================================================
FILE: .github/workflows/build.yml
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
#
# Intentionally has a general name.
# because the test status check created in GitHub Actions
# currently randomly picks any associated workflow.
# So, the name was changed to make sense in that context too.
# See also https://github.community/t/specify-check-suite-when-creating-a-checkrun/118380/10
name: "On pull requests"
on:
workflow_dispatch:
pull_request:
push:
branches:
- master
env:
SPARK_VERSION: '4.0.0'
HADOOP_VERSION: '3'
permissions:
# Required: allow read access to the content for analysis.
contents: read
# Optional: allow read access to pull request. Use with `only-new-issues` option.
pull-requests: read
# Optional: allow write access to checks to allow the action to annotate code in the PR.
checks: write
jobs:
build:
name: Build & Test Client
runs-on: ubuntu-latest
steps:
- name: Checkout Repository
uses: actions/checkout@v4
with:
submodules: recursive
- uses: actions/setup-go@v5
name: Setup Go
with:
go-version-file: 'go.mod'
- uses: actions/setup-python@v5
with:
python-version: '3.10'
- uses: actions/setup-java@v4
with:
java-version: '17'
distribution: zulu
- name: Cache Spark Installation
uses: actions/cache@v4
id: cache
with:
key: v2-spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
path: |
/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
- name: Setup SBT
uses: sbt/setup-sbt@v1
- name: Setup Apache Spark
if: steps.cache.outputs.cache-hit != 'true'
run: |
set -x
echo "Apache Spark is not installed"
# Access the directory.
mkdir -p ~/deps/
wget -q https://dlcdn.apache.org/spark/spark-${{ env.SPARK_VERSION }}/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz
tar -xzf spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz -C ~/deps/
# Delete the old file
rm spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}.tgz
ls -lah ~/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
du -hs ~/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
# Setup the Environment Variables
echo "Apache Spark is ready to use"
echo "SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}" >> "$GITHUB_ENV"
- name: Run Build & Test
run: |
go mod download -x
# Remove dependency on gen until Spark 4 has the fix for the pipelines.proto
# make gen
make
make test
- name: Run Example Spark Submit Application
run: |
export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
make
cd java
sbt publishLocal
./run.sh ../cmd/spark-connect-example-spark-session/spark-connect-example-spark-session
- name: Run Integration Test
run: |
export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
# Remove dependency on gen until Spark 4 has the fix for the pipelines.proto
# make gen
make && make integration
- name: Run Code Coverage
run: |
export SPARK_HOME=/home/runner/deps/spark-${{ env.SPARK_VERSION }}-bin-hadoop${{ env.HADOOP_VERSION }}
make coverage
# Disable the code coverage check for now until https://github.com/PaloAltoNetworks/cov/issues/8 is fixed.
# - uses: PaloAltoNetworks/cov@3.0.0
# with:
# cov_mode: coverage
# main_branch: master
# cov_threshold: 60
- name: golangci-lint
uses: golangci/golangci-lint-action@v8
with:
version: v2.1
================================================
FILE: .gitignore
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
# All generated files
internal/generated.out
# Ignore Coverage Files
coverage*
cov.report
# Ignore IDE files
.idea/
# Ignore binaries
cmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client
cmd/spark-connect-example-spark-session/spark-connect-example-spark-session
target
lib
deps
.DS_Store
================================================
FILE: .gitmodules
================================================
[submodule "spark"]
path = sparksrc
url = https://github.com/apache/spark.git
[submodule "sparksrc"]
branch = branch-4.0
================================================
FILE: .golangci.yml
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
version: "2"
linters:
exclusions:
generated: lax
presets:
- comments
- common-false-positives
- legacy
- std-error-handling
paths:
- third_party$
- builtin$
- examples$
formatters:
enable:
- gofumpt
exclusions:
generated: lax
paths:
- third_party$
- builtin$
- examples$
================================================
FILE: CONTRIBUTING.md
================================================
## Contributing to Spark
*Before opening a pull request*, review the
[Contributing to Spark guide](https://spark.apache.org/contributing.html).
It lists steps that are required before creating a PR. In particular, consider:
- Is the change important and ready enough to ask the community to spend time reviewing?
- Have you searched for existing, related JIRAs and pull requests?
- Is this a new feature that can stand alone as a [third party project](https://spark.apache.org/third-party-projects.html) ?
- Is the change being proposed clearly explained and motivated?
When you contribute code, you affirm that the contribution is your original work and that you
license the work to the project under the project's open source license. Whether or not you
state this explicitly, by submitting any copyrighted material via pull request, email, or
other means you agree to license the material under the project's open source license and
warrant that you have the legal authority to do so.
### Code Style and Checks
When submitting code we use a number of checks in our continous integration system to ensure
a consitent style and adherence to license rules. You can run these checks locally by running:
```bash
make check
```
This requires the following tools to be present in your `PATH`:
1. Java for checking license headers
2. [gofumpt](https://github.com/mvdan/gofumpt) for formatting Go code
3. [golangci-lint](https://golangci-lint.run/) for linting Go code
### Running Tests
To run the tests locally, you can run:
```bash
make test
```
This will run the unit tests. If you want to run the integration tests, you can run (you
need to set environment variable `SPARK_HOME` pointing to existing directory with unpacked
Apache Spark 3.5+ distribution):
```bash
make integration
```
Lastly, if you want to run all tests (unit and integration) and generate the coverage
analysis, you can run:
```bash
make fulltest
```
The output of the coverage analysis will be in the `coverage.out` file. An HTML version of
the coverage report is generated and accessible at `coverage.html`.
### How to write tests
Please make sure that you have proper testing for the new code your adding. As part of the
code base we started to add mocks that allow you to simulate a lot of the necessary API
and don't require a running Spark instance.
`mock.ProtoClient` is a mock implementation of the `SparkConnectService_ExecutePlanClient`
interface which is the server-side stream of messages coming as a response from the server.
`testutils.NewConnectServiceClientMock` will create a mock client that implements the
`SparkConnectServiceClient` interface.
The combination of these two mocks allows you to test the client side of the code without
having to connect to Spark.
### What to contribute
We welcome contributions of all kinds to the `spark-connect-go` project. Some examples of
contributions are providing implementations of functionality that is missing in the Go
implementation. Some examples are, but are not limited to:
* Adding an existing feature of the DataFrame API in Golang.
* Adding support for a builtin function in the Spark API in Golang.
* Improving error handling in the client.
If you are unsure about whether a contribution is a good fit, feel free to open an issue
in the Apache Spark Jira.
================================================
FILE: LICENSE
================================================
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
================================================
FILE: Makefile
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
FIRST_GOPATH := $(firstword $(subst :, ,$(GOPATH)))
PKGS := $(shell go list ./... | grep -v /tests | grep -v /xcpb | grep -v /gpb | grep -v /generated)
GOFILES_NOVENDOR := $(shell find . -name vendor -prune -o -type f -name '*.go' -not -name '*.pb.go' -print)
GOFILES_BUILD := $(shell find . -type f -name '*.go' -not -name '*_test.go')
PROTOFILES := $(shell find . -name vendor -prune -o -type f -name '*.proto' -print)
ALLGOFILES := $(shell find . -type f -name '*.go' -not -name '*.pb.go')
DATE := $(shell date -u -d "@$(SOURCE_DATE_EPOCH)" '+%FT%T%z' 2>/dev/null || date -u '+%FT%T%z')
BUILDFLAGS_NOPIE :=
BUILDFLAGS ?= $(BUILDFLAGS_NOPIE) -buildmode=pie
TESTFLAGS ?=
PWD := $(shell pwd)
PREFIX ?= $(GOPATH)
BINDIR ?= $(PREFIX)/bin
GO := go
GOOS ?= $(shell go version | cut -d' ' -f4 | cut -d'/' -f1)
GOARCH ?= $(shell go version | cut -d' ' -f4 | cut -d'/' -f2)
TAGS ?= netgo
SHELL = bash
GOFUMPT_SPLIT_LONG_LINES := on
## Build tools
BUF := $(GO) run github.com/bufbuild/buf/cmd/buf@v1.26.1
BINARIES := cmd/spark-connect-example-spark-session/spark-connect-example-spark-session cmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client
# Define the location of SPARK_HOME because we need that to depend on the build paths
MAKEFILE_DIR:=$(shell dirname $(realpath $(firstword $(MAKEFILE_LIST))))
PROTO_SRC = $(shell find internal/generated -type f -name *.proto )
OK := $(shell tput setaf 6; echo ' [OK]'; tput sgr0;)
all: build
build: $(BUILD_OUTPUT) $(BINARIES)
cmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client: $(GOFILES_BUILD)
@echo ">> BUILD, output = $@"
@cd $(dir $@) && $(GO) build -o $(notdir $@) $(BUILDFLAGS)
@printf '%s\n' '$(OK)'
cmd/spark-connect-example-spark-session/spark-connect-example-spark-session: $(GOFILES_BUILD)
@echo ">> BUILD, output = $@"
@cd $(dir $@) && $(GO) build -o $(notdir $@) $(BUILDFLAGS)
@printf '%s\n' '$(OK)'
internal/generated.out:
@echo -n ">> BUILD, output = $@"
$(BUF) generate --debug -vvv
@touch internal/generated.out
@printf '%s\n' '$(OK)'
gen: internal/generated.out
$(GOFILES_BUILD):
$(BUILD_OUTPUT): $(GOFILES_BUILD)
@echo -n ">> BUILD, output = $@"
@$(GO) build -o $@ $(BUILDFLAGS)
@printf '%s\n' '$(OK)'
lint: $(BUILD_OUTPUT)
@golangci-lint run
fmt:
@echo -n ">> glongci-lint: fix"
env GOFUMPT_SPLIT_LONG_LINES=$(GOFUMPT_SPLIT_LONG_LINES) golangci-lint run --fix
test: $(BUILD_OUTPUT)
@echo ">> TEST, \"verbose\""
@$(foreach pkg, $(PKGS),\
@echo -n " ";\
$(GO) test -v -run '(Test|Example)' $(BUILDFLAGS) $(TESTFLAGS) $(pkg) || exit 1)
coverage: $(BUILD_OUTPUT)
@echo ">> TEST, \"coverage\""
@$(GO) test -cover -coverprofile=coverage.out -covermode=atomic -coverpkg=./spark/...,./internal/tests/... ./spark/... ./internal/tests/...
@$(GO) tool cover -html=coverage.out -o coverage.html
integration: $(BUILD_OUTPUT)
@echo ">> TEST, \"integration\""
@$(GO) test ./internal/tests/...
check:
@echo -n ">> CHECK"
./dev/check-license
@echo -n ">> glongci-lint: "
env GOFUMPT_SPLIT_LONG_LINES=$(GOFUMPT_SPLIT_LONG_LINES) golangci-lint run
clean:
@echo -n ">> CLEAN"
@$(GO) clean -i ./...
@rm -f ./coverage-all.html
@rm -f ./coverage-all.out
@rm -f ./coverage.out
@find . -type f -name "coverage.out" -delete
@printf '%s\n' '$(OK)'
cleangen:
@rm -rf ./internal/generated
@rm -f ./internal/generated.out
cleanall: clean cleangen
================================================
FILE: NOTICE
================================================
Apache Spark
Copyright 2014 and onwards The Apache Software Foundation.
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
================================================
FILE: README.md
================================================
# Apache Spark Connect Client for Golang
This project houses the **experimental** client for [Spark
Connect](https://spark.apache.org/docs/latest/spark-connect-overview.html) for
[Apache Spark](https://spark.apache.org/) written in [Golang](https://go.dev/).
## Current State of the Project
Currently, the Spark Connect client for Golang is highly experimental and should
not be used in any production setting. In addition, the PMC of the Apache Spark
project reserves the right to withdraw and abandon the development of this project
if it is not sustainable.
## Getting started
This section explains how to run Spark Connect Go locally.
Step 1: Install Golang: https://go.dev/doc/install.
Step 2: Ensure you have installed `buf CLI` installed, [more info here](https://buf.build/docs/installation/)
Step 3: Run the following commands to setup the Spark Connect client.
Building with Spark in case you need to re-generate the source files from the proto sources.
```
git clone https://github.com/apache/spark-connect-go.git
git submodule update --init --recursive
make gen && make test
```
Building without Spark
```
git clone https://github.com/apache/spark-connect-go.git
make && make test
```
Step 4: Setup the Spark Driver on localhost.
1. [Download Spark distribution](https://spark.apache.org/downloads.html) (4.0.0+), unzip the package.
2. Start the Spark Connect server with the following command (make sure to use a package version that matches your Spark distribution):
```
sbin/start-connect-server.sh
```
Step 5: Run the example Go application.
```
go run cmd/spark-connect-example-spark-session/main.go
```
## Runnning Spark Connect Go Application in a Spark Cluster
To 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.
See the guide here: [Sample Spark-Submit Wrapper](java/README.md).
## How to write Spark Connect Go Application in your own project
See [Quick Start Guide](quick-start.md)
## High Level Design
The overall goal of the design is to find a good balance of principle of the least surprise for
develoeprs that are familiar with the APIs of Apache Spark and idiomatic Go usage. The high-level
structure of the packages follows roughly the PySpark giudance but with Go idioms.
## Contributing
Please review the [Contribution to Spark guide](https://spark.apache.org/contributing.html)
for information on how to get started contributing to the project.
================================================
FILE: buf.gen.yaml
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
version: v1
plugins:
- plugin: buf.build/protocolbuffers/go:v1.30.0
out: .
- plugin: buf.build/grpc/go:v1.3.0
out: .
================================================
FILE: buf.work.yaml
================================================
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
version: v1
directories:
- sparksrc/sql/connect/common/src/main/protobuf
================================================
FILE: cmd/spark-connect-example-raw-grpc-client/main.go
================================================
//
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package main
import (
"context"
"flag"
"log"
"time"
proto "github.com/apache/spark-connect-go/internal/generated"
"github.com/google/uuid"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
var remote = flag.String("remote", "localhost:15002", "the remote address of Spark Connect server to connect to")
func main() {
opts := []grpc.DialOption{
grpc.WithTransportCredentials(insecure.NewCredentials()),
}
conn, err := grpc.NewClient(*remote, opts...)
if err != nil {
log.Fatalf("Failed: %s", err)
}
defer conn.Close()
client := proto.NewSparkConnectServiceClient(conn)
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
defer cancel()
configRequest := proto.ConfigRequest{
SessionId: uuid.NewString(),
Operation: &proto.ConfigRequest_Operation{
OpType: &proto.ConfigRequest_Operation_GetAll{
GetAll: &proto.ConfigRequest_GetAll{},
},
},
}
configResponse, err := client.Config(ctx, &configRequest)
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("configResponse: %v", configResponse)
}
================================================
FILE: cmd/spark-connect-example-spark-session/main.go
================================================
//
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package main
import (
"context"
"flag"
"fmt"
"log"
"github.com/apache/spark-connect-go/spark/sql/types"
"github.com/apache/spark-connect-go/spark/sql/functions"
"github.com/apache/spark-connect-go/spark/sql"
"github.com/apache/spark-connect-go/spark/sql/utils"
)
var (
remote = flag.String("remote", "sc://localhost:15002",
"the remote address of Spark Connect server to connect to")
filedir = flag.String("filedir", "/tmp",
"the root directory to save the files generated by this example program")
)
func main() {
flag.Parse()
ctx := context.Background()
spark, err := sql.NewSessionBuilder().Remote(*remote).Build(ctx)
if err != nil {
log.Fatalf("Failed: %s", err)
}
defer utils.WarnOnError(spark.Stop, func(err error) {})
df, err := spark.Sql(ctx, "select id from range(100)")
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, _ = df.FilterByString(ctx, "id < 10")
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, err = spark.Sql(ctx, "select * from range(100)")
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, _ = df.Filter(ctx, functions.Col("id").Lt(functions.Expr("10")))
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, _ = spark.Sql(ctx, "select * from range(100)")
df, err = df.Filter(ctx, functions.Col("id").Lt(functions.Lit(types.Int64(20))))
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, err = spark.Sql(ctx, "select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count")
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("DataFrame from sql: select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count")
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
schema, err := df.Schema(ctx)
if err != nil {
log.Fatalf("Failed: %s", err)
}
for _, f := range schema.Fields {
log.Printf("Field in dataframe schema: %s - %s", f.Name, f.DataType.TypeName())
}
rows, err := df.Collect(ctx)
if err != nil {
log.Fatalf("Failed: %s", err)
}
schema, err = df.Schema(ctx)
if err != nil {
log.Fatalf("Failed: %s", err)
}
for _, f := range schema.Fields {
log.Printf("Field in row: %s - %s", f.Name, f.DataType.TypeName())
}
for _, row := range rows {
log.Printf("Row: %v", row)
}
err = df.Writer().Mode("overwrite").
Format("parquet").
Save(ctx, fmt.Sprintf("file://%s/spark-connect-write-example-output.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, err = spark.Read().Format("parquet").
Load(fmt.Sprintf("file://%s/spark-connect-write-example-output.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("DataFrame from reading parquet")
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.CreateTempView(ctx, "view1", true, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
df, err = spark.Sql(ctx, "select count, word from view1 order by count")
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("DataFrame from sql: select count, word from view1 order by count")
err = df.Show(ctx, 100, false)
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("Repartition with one partition")
df, err = df.Repartition(ctx, 1, nil)
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.Writer().Mode("overwrite").
Format("parquet").
Save(ctx, fmt.Sprintf("file://%s/spark-connect-write-example-output-one-partition.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("Repartition with two partitions")
df, err = df.Repartition(ctx, 2, nil)
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.Writer().Mode("overwrite").
Format("parquet").
Save(ctx, fmt.Sprintf("file://%s/spark-connect-write-example-output-two-partition.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("Repartition with columns")
df, err = df.Repartition(ctx, 0, []string{"word", "count"})
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.Writer().Mode("overwrite").
Format("parquet").
Save(ctx, fmt.Sprintf("file://%s/spark-connect-write-example-output-repartition-with-column.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
log.Printf("Repartition by range with columns")
df, err = df.RepartitionByRange(ctx, 0, functions.Col("word").Desc())
if err != nil {
log.Fatalf("Failed: %s", err)
}
err = df.Writer().Mode("overwrite").
Format("parquet").
Save(ctx, fmt.Sprintf("file:///%s/spark-connect-write-example-output-repartition-by-range-with-column.parquet", *filedir))
if err != nil {
log.Fatalf("Failed: %s", err)
}
}
================================================
FILE: dev/.rat-excludes
================================================
.gitignore
.gitmodules
.gitattributes
.project
coverage*
LICENSE
NOTICE
TAGS
RELEASE
.*md
.rat-excludes
sparksrc
target
generated.out
go.sum
deps
cov.report
build.properties
spark-connect-go.code-workspace
================================================
FILE: dev/README.md
================================================
# Release Script for Apache Spark Connect Go
This directory contains the release automation script for the Apache Spark Connect Go project.
## Prerequisites
1. **Python Environment**: Create a virtual environment and install dependencies:
```bash
python -m venv venv
source venv/bin/activate # On Windows: venv\Scripts\activate
pip install -r requirements.txt
```
2. **GitHub Token**: Create a GitHub personal access token with the following permissions:
- `repo` (Full control of private repositories)
- `write:packages` (Upload packages to GitHub Package Registry)
3. **GPG Key**: Ensure you have a GPG key set up for signing:
```bash
# List available keys
gpg --list-secret-keys
# If you don't have a key, create one
gpg --gen-key
```
## Usage
```bash
./release.py --tag <new_tag> --prev-tag <previous_tag> --commit <commit_sha> --gpg-user <gpg_user_id> [options]
```
### Required Arguments
- `--tag`: The new tag version (e.g., `v0.2.0`)
- `--prev-tag`: The previous tag version for generating release notes (e.g., `v0.1.0`)
- `--commit`: The commit SHA that the tag should point to
- `--gpg-user`: Your GPG user ID for signing (email or key ID)
### Optional Arguments
- `--prerelease`: Mark the release as a pre-release
- `--repo`: GitHub repository in format `owner/name` (default: `apache/spark-connect-go`)
- `--token`: GitHub token (alternatively set `GITHUB_TOKEN` environment variable)
### Environment Variables
- `GITHUB_TOKEN`: GitHub personal access token
## Example Usage
```bash
# Set GitHub token
export GITHUB_TOKEN=ghp_your_token_here
# Create a regular release
./release.py \
--tag v0.2.0 \
--prev-tag v0.1.0 \
--commit abc123def456 \
--gpg-user your.email@example.com
# Create a pre-release
./release.py \
--tag v0.2.0-rc1 \
--prev-tag v0.1.0 \
--commit abc123def456 \
--gpg-user your.email@example.com \
--prerelease
```
## What the Script Does
1. **Creates and pushes tag**: Creates a Git tag at the specified commit and pushes it to GitHub
2. **Generates release notes**: Automatically creates initial release notes from commits between tags
3. **Interactive input**: Prompts you to enter/modify the release description
4. **Creates GitHub release**: Creates a draft release on GitHub
5. **Downloads artifacts**: Downloads the automatically generated source archives (.tar.gz, .zip)
6. **Signs artifacts**: Creates detached GPG signatures for each artifact
7. **Verifies signatures**: Confirms that all signatures are valid
8. **Uploads signatures**: Uploads the signature files to the GitHub release
## Output
The script creates:
- A new Git tag pushed to GitHub
- A draft GitHub release with:
- Source code archives (automatically generated by GitHub)
- Detached GPG signatures (.asc files)
- Release notes based on commits
## Security Notes
- All artifacts are signed with your GPG key
- Signatures are verified before upload
- The release is created as a draft first for review
- Your GPG key must be available in your keyring
## Troubleshooting
### GPG Issues
```bash
# If GPG signing fails, check your key
gpg --list-secret-keys
# Test signing
echo "test" | gpg --clearsign --local-user your.email@example.com
```
### GitHub API Issues
- Ensure your token has the correct permissions
- Check rate limits if requests fail
- Verify repository access
### Git Issues
- Ensure you're in the correct repository directory
- Check that the commit SHA exists
- Verify you have push permissions to the repository
================================================
FILE: dev/check-license
================================================
#!/usr/bin/env bash
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
acquire_rat_jar () {
URL="${DEFAULT_ARTIFACT_REPOSITORY:-https://repo1.maven.org/maven2/}org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar"
JAR="$rat_jar"
# Download rat launch jar if it hasn't been downloaded yet
if [ ! -f "$JAR" ]; then
# Download
printf "Attempting to fetch rat\n"
JAR_DL="${JAR}.part"
if [ $(command -v curl) ]; then
curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR"
elif [ $(command -v wget) ]; then
wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR"
else
printf "You do not have curl or wget installed, please install rat manually.\n"
exit -1
fi
fi
unzip -tq "$JAR" &> /dev/null
if [ $? -ne 0 ]; then
# We failed to download
rm "$JAR"
printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n"
exit -1
fi
}
# Go to the Spark project root directory
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
cd "$FWDIR"
if test -x "$JAVA_HOME/bin/java"; then
declare java_cmd="$JAVA_HOME/bin/java"
else
declare java_cmd=java
fi
export RAT_VERSION=0.15
export rat_jar="$FWDIR"/lib/apache-rat-${RAT_VERSION}.jar
mkdir -p "$FWDIR"/lib
[[ -f "$rat_jar" ]] || acquire_rat_jar || {
echo "Download failed. Obtain the rat jar manually and place it at $rat_jar"
exit 1
}
mkdir -p target
$java_cmd -jar "$rat_jar" -E "$FWDIR"/dev/.rat-excludes -d "$FWDIR" > target/rat-results.txt
if [ $? -ne 0 ]; then
echo "RAT exited abnormally"
exit 1
fi
ERRORS="$(cat target/rat-results.txt | grep -e "??")"
if test ! -z "$ERRORS"; then
echo "Could not find Apache license headers in the following files:"
echo "$ERRORS"
exit 1
else
echo -e "RAT checks passed."
fi
================================================
FILE: dev/gen.py
================================================
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
# This is a basic script to generate the builtin functions based on the
# currently available PySpark installation.
# Simply call the script as follows:
#
# python gen.py > spark/client/functions/generated.go
import pyspark.sql.connect.functions as F
import inspect
import typing
import types
def normalize(input: str) -> str:
vals = [x[0].upper() + x[1:] for x in input.split("_")]
return "".join(vals)
print("""
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package functions
import "github.com/apache/spark-connect-go/spark/sql/column"
""")
for fun in F.__dict__:
if fun.startswith("_"):
continue
if not callable(F.__dict__[fun]):
continue
if "pyspark.sql.connect.functions" not in F.__dict__[fun].__module__:
continue
if fun == "expr" or fun == "col" or fun == "column" or fun == "lit":
continue
# Ignore the aliases of the old distinct.
if "Distinct" in fun:
continue
sig = inspect.signature(F.__dict__[fun])
# Ignore all functions that take callables as parameters
has_callable = False
for p in sig.parameters:
param = sig.parameters[p]
if "Callable" in str(param.annotation):
has_callable = True
break
if has_callable:
print(f"// TODO: {fun}: {sig}")
print()
continue
if "udf" in fun.lower():
print(f"// Ignore UDF: {fun}: {sig}")
print()
continue
if "udt" in fun.lower():
print(f"// Ignore UDT: {fun}: {sig}")
print()
continue
# Convert parameters into Golang
res_params = []
conversions = []
args = []
valid = True
for p in sig.parameters:
param = sig.parameters[p]
if param.annotation == inspect.Parameter.empty:
res_params.append(f"{p} interface{{}}")
args.append(p)
elif param.kind == inspect.Parameter.VAR_POSITIONAL and param.annotation == "ColumnOrName":
res_params.append(f"{p} ...column.Column")
conversions.append("vals := make([]column.Column, 0)")
for x in args:
conversions.append(f"vals = append(vals, {x})")
conversions.append(f"vals = append(vals, {p}...)")
args = ["vals..."]
elif type(param.annotation) == str and str(param.annotation) == "ColumnOrName" and param.kind != inspect.Parameter.VAR_POSITIONAL and param.kind != inspect.Parameter.VAR_KEYWORD:
res_params.append(f"{p} column.Column")
args.append(p)
elif len(typing.get_args(param.annotation)) > 1 and typing.ForwardRef("ColumnOrName") in typing.get_args(param.annotation):
# Find the parameter with ColumnOrName
tmp = [x for x in typing.get_args(param.annotation) if typing.ForwardRef("ColumnOrName") == x]
assert len(tmp) == 1
res_params.append(f"{p} column.Column")
args.append(p)
elif param.annotation == str or typing.get_args(param.annotation) == (str, types.NoneType):
res_params.append(f"{p} string")
conversions.append(f"lit_{p} := StringLit({p})")
args.append(f"lit_{p}")
elif param.annotation == int or typing.get_args(param.annotation) == (int, types.NoneType):
res_params.append(f"{p} int64")
conversions.append(f"lit_{p} := Int64Lit({p})")
args.append(f"lit_{p}")
elif param.annotation == float or typing.get_args(param.annotation) == (float, types.NoneType):
res_params.append(f"{p} float64")
conversions.append(f"lit_{p} := Float64Lit({p})")
args.append(f"lit_{p}")
else:
valid = False
break
if not valid:
print(f"// TODO: {fun}: {sig}")
print()
else:
name = normalize(fun)
# Generate the doc string
if F.__dict__[fun].__doc__ is not None:
lines = list(map(str.lstrip, F.__dict__[fun].__doc__.split("\n")))
pos = list(map(lambda x: x.startswith("..") or x.startswith("Parameters"), lines)).index(True)
lines = "\n".join(lines[:pos]).strip().split("\n")
lines[0] = name + " - " + lines[0]
lines = ["// " + l for l in lines]
doc = "\n".join(lines) + "\n//"
print(doc)
print(f"// {name} is the Golang equivalent of {fun}: {sig}")
print(f"func {name}({', '.join(res_params)}) column.Column {{")
for c in conversions:
print(f" {c}")
print(f" return column.NewColumn(column.NewUnresolvedFunctionWithColumns(\"{fun}\", {', '.join(args)}))")
print(f"}}")
print()
================================================
FILE: dev/release.py
================================================
#!/usr/bin/env python3
"""
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
"""
import argparse
import os
import subprocess
import sys
import tempfile
import requests
from pathlib import Path
from typing import List, Dict, Any
import git
from github import Github
def run_command(cmd: List[str], cwd: str = None, check: bool = True) -> subprocess.CompletedProcess:
"""Run a shell command and return the result."""
print(f"Running: {' '.join(cmd)}")
result = subprocess.run(cmd, cwd=cwd, capture_output=True, text=True, check=False)
if result.returncode != 0 and check:
print(f"Command failed with return code {result.returncode}")
print(f"STDOUT: {result.stdout}")
print(f"STDERR: {result.stderr}")
sys.exit(1)
return result
def get_commits_between_tags(repo_path: str, previous_tag: str, commit_sha: str) -> List[Dict[str, str]]:
"""Get commits between previous tag and current commit."""
try:
repo = git.Repo(repo_path)
# Get commits from previous tag to current commit
commits = list(repo.iter_commits(f"{previous_tag}..{commit_sha}"))
commit_info = []
for commit in commits:
commit_info.append({
'sha': commit.hexsha[:8], # Short commit ID
'author': commit.author.name,
'message': commit.message.split('\n')[0] # Subject line only
})
return commit_info
except Exception as e:
print(f"Error getting commits: {e}")
return []
def create_release_notes(commits: List[Dict[str, str]]) -> str:
"""Create initial release notes from commits."""
if not commits:
return "## Changes\n\nNo commits found between releases.\n"
notes = "## Changes\n\n"
for commit in commits:
notes += f"* {commit['sha']} - {commit['message']} ({commit['author']})\n"
return notes
def verify_gpg_key(gpg_user: str) -> bool:
"""Verify that the GPG key exists and can be used for signing."""
try:
result = run_command(['gpg', '--list-secret-keys', gpg_user], check=False)
return result.returncode == 0
except Exception:
return False
def sign_file(file_path: str, gpg_user: str) -> str:
"""Create a detached GPG signature for a file."""
signature_path = f"{file_path}.asc"
cmd = [
'gpg',
'--local-user', gpg_user,
'--armor',
'--detach-sign',
file_path
]
run_command(cmd)
if not os.path.exists(signature_path):
raise RuntimeError(f"Signature file {signature_path} was not created")
return signature_path
def verify_signature(file_path: str, signature_path: str) -> bool:
"""Verify a GPG signature."""
try:
result = run_command(['gpg', '--verify', signature_path, file_path], check=False)
return result.returncode == 0
except Exception:
return False
def download_file(url: str, local_path: str):
"""Download a file from URL to local path."""
print(f"Downloading {url} to {local_path}")
response = requests.get(url, stream=True)
response.raise_for_status()
with open(local_path, 'wb') as f:
for chunk in response.iter_content(chunk_size=8192):
f.write(chunk)
def upload_release_asset(release, file_path: str):
"""Upload a file as a release asset."""
print(f"Uploading {file_path} to release")
filename = os.path.basename(file_path)
# Use the release object's upload_asset method
# PyGithub expects: upload_asset(path, label=None, content_type=None, name=None)
release.upload_asset(file_path, label=filename, name=filename)
def main():
parser = argparse.ArgumentParser(description='Create and sign Apache Spark Connect Go release')
parser.add_argument('--tag', required=True, help='New tag version (e.g., v0.2.0)')
parser.add_argument('--prev-tag', required=True, help='Previous tag version (e.g., v0.1.0)')
parser.add_argument('--commit', required=True, help='Commit SHA for the tag')
parser.add_argument('--gpg-user', required=True, help='GPG user ID for signing')
parser.add_argument('--prerelease', action='store_true', help='Mark as pre-release')
parser.add_argument('--repo', default='apache/spark-connect-go', help='GitHub repository (owner/name)')
parser.add_argument('--token', help='GitHub token (or set GITHUB_TOKEN env var)')
args = parser.parse_args()
# Get GitHub token
github_token = args.token or os.environ.get('GITHUB_TOKEN')
if not github_token:
print("Error: GitHub token is required. Use --token or set GITHUB_TOKEN environment variable.")
sys.exit(1)
# Verify GPG key exists
if not verify_gpg_key(args.gpg_user):
print(f"Error: GPG key for user '{args.gpg_user}' not found or not usable")
sys.exit(1)
# Initialize GitHub client
github_client = Github(github_token)
repo = github_client.get_repo(args.repo)
print(f"Creating release for {args.repo}")
print(f"Tag: {args.tag}")
print(f"Commit: {args.commit}")
print(f"Previous tag: {args.prev_tag}")
print(f"GPG user: {args.gpg_user}")
print(f"Pre-release: {args.prerelease}")
# Step 1: Create and push tag
print("\n=== Step 1: Creating and pushing tag ===")
repo_path = os.getcwd()
try:
local_repo = git.Repo(repo_path)
# Create tag
new_tag = local_repo.create_tag(args.tag, ref=args.commit, message=f"Release {args.tag}")
print(f"Created tag {args.tag} at commit {args.commit}")
# Push tag
origin = local_repo.remote('origin')
origin.push(new_tag)
print(f"Pushed tag {args.tag} to GitHub")
except Exception as e:
print(f"Error creating/pushing tag: {e}")
sys.exit(1)
# Step 2: Get commits for release notes
print("\n=== Step 2: Generating release notes ===")
commits = get_commits_between_tags(repo_path, args.prev_tag, args.commit)
initial_release_notes = create_release_notes(commits)
# Step 3: Prompt user for release description
print("\n=== Step 3: Release description ===")
print("Initial release notes based on commits:")
print(initial_release_notes)
print("\nPlease enter the final release description (press Ctrl+D when done):")
lines = []
try:
while True:
line = input()
lines.append(line)
except EOFError:
pass
# Join the lines and add the initial release notes
final_release_notes = '\n'.join(lines).strip()
spacer = "\n\n" if final_release_notes else ""
final_release_notes += spacer + initial_release_notes
# Step 4: Create GitHub release
print("\n=== Step 4: Creating GitHub release ===")
try:
release = repo.create_git_release(
tag=args.tag,
name=f"Release {args.tag}",
message=final_release_notes,
draft=True,
prerelease=args.prerelease
)
print(f"Created draft release: {release.html_url}")
except Exception as e:
print(f"Error creating release: {e}")
sys.exit(1)
# Step 5: Download release artifacts
print("\n=== Step 5: Downloading release artifacts ===")
# GitHub automatically creates source archives
artifacts = [
f"{args.tag}.tar.gz",
f"{args.tag}.zip"
]
with tempfile.TemporaryDirectory() as temp_dir:
downloaded_files = []
for artifact in artifacts:
# Construct download URL for source archive
download_url = f"https://github.com/{args.repo}/archive/refs/tags/{artifact}"
local_file = os.path.join(temp_dir, f"spark-connect-go-{artifact}")
try:
download_file(download_url, local_file)
downloaded_files.append(local_file)
except Exception as e:
print(f"Error downloading {artifact}: {e}")
continue
if not downloaded_files:
print("Error: No artifacts were downloaded")
sys.exit(1)
# Step 6: Sign artifacts
print("\n=== Step 6: Signing artifacts ===")
signatures = []
for file_path in downloaded_files:
try:
print(f"Signing {os.path.basename(file_path)}")
signature_path = sign_file(file_path, args.gpg_user)
signatures.append(signature_path)
print(f"Created signature: {os.path.basename(signature_path)}")
except Exception as e:
print(f"Error signing {file_path}: {e}")
continue
# Step 7: Verify signatures
print("\n=== Step 7: Verifying signatures ===")
for i, file_path in enumerate(downloaded_files):
if i < len(signatures):
signature_path = signatures[i]
if verify_signature(file_path, signature_path):
print(f"✓ Signature verified for {os.path.basename(file_path)}")
else:
print(f"✗ Signature verification failed for {os.path.basename(file_path)}")
sys.exit(1)
# Step 8: Upload signatures to release
print("\n=== Step 8: Uploading signatures to release ===")
for signature_path in signatures:
try:
upload_release_asset(release, signature_path)
print(f"Uploaded {os.path.basename(signature_path)}")
except Exception as e:
print(f"Error uploading {signature_path}: {e}")
continue
print(f"\n=== Release created successfully ===")
print(f"Release URL: {release.html_url}")
print(f"Tag: {args.tag}")
print(f"Status: Draft")
print(f"Pre-release: {args.prerelease}")
print("\nNext steps:")
print("1. Review the release on GitHub")
print("2. Test the release artifacts")
print("3. Publish the release when ready")
if __name__ == '__main__':
main()
================================================
FILE: dev/requirements.txt
================================================
requests>=2.28.0
PyGithub>=1.58.0
gitpython>=3.1.30
================================================
FILE: go.mod
================================================
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
module github.com/apache/spark-connect-go
go 1.23.2
require (
github.com/apache/arrow-go/v18 v18.4.0
github.com/go-errors/errors v1.5.1
github.com/google/uuid v1.6.0
github.com/stretchr/testify v1.10.0
google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7
google.golang.org/grpc v1.75.0
google.golang.org/protobuf v1.36.7
)
require (
cloud.google.com/go/compute/metadata v0.7.0 // indirect
github.com/kr/pretty v0.3.0 // indirect
golang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0 // indirect
golang.org/x/net v0.41.0 // indirect
golang.org/x/sync v0.15.0 // indirect
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect
)
require (
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect
github.com/goccy/go-json v0.10.5 // indirect
github.com/google/flatbuffers v25.2.10+incompatible // indirect
github.com/klauspost/compress v1.18.0 // indirect
github.com/klauspost/cpuid/v2 v2.2.11 // indirect
github.com/pierrec/lz4/v4 v4.1.22 // indirect
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
github.com/zeebo/xxh3 v1.0.2 // indirect
golang.org/x/mod v0.25.0 // indirect
golang.org/x/oauth2 v0.30.0
golang.org/x/sys v0.33.0 // indirect
golang.org/x/text v0.26.0 // indirect
golang.org/x/tools v0.34.0 // indirect
golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
)
================================================
FILE: go.sum
================================================
cloud.google.com/go/compute/metadata v0.7.0 h1:PBWF+iiAerVNe8UCHxdOt6eHLVc3ydFeOCw78U8ytSU=
cloud.google.com/go/compute/metadata v0.7.0/go.mod h1:j5MvL9PprKL39t166CoB1uVHfQMs4tFQZZcKwksXUjo=
github.com/andybalholm/brotli v1.2.0 h1:ukwgCxwYrmACq68yiUqwIWnGY0cTPox/M94sVwToPjQ=
github.com/andybalholm/brotli v1.2.0/go.mod h1:rzTDkvFWvIrjDXZHkuS16NPggd91W3kUSvPlQ1pLaKY=
github.com/apache/arrow-go/v18 v18.4.0 h1:/RvkGqH517iY8bZKc4FD5/kkdwXJGjxf28JIXbJ/oB0=
github.com/apache/arrow-go/v18 v18.4.0/go.mod h1:Aawvwhj8x2jURIzD9Moy72cF0FyJXOpkYpdmGRHcw14=
github.com/apache/thrift v0.22.0 h1:r7mTJdj51TMDe6RtcmNdQxgn9XcyfGDOzegMDRg47uc=
github.com/apache/thrift v0.22.0/go.mod h1:1e7J/O1Ae6ZQMTYdy9xa3w9k+XHWPfRvdPyJeynQ+/g=
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM=
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/go-errors/errors v1.5.1 h1:ZwEMSLRCapFLflTpT7NKaAc7ukJ8ZPEjzlxt8rPN8bk=
github.com/go-errors/errors v1.5.1/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og=
github.com/go-logr/logr v1.4.3 h1:CjnDlHq8ikf6E492q6eKboGOC0T8CDaOvkHCIg8idEI=
github.com/go-logr/logr v1.4.3/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
github.com/goccy/go-json v0.10.5 h1:Fq85nIqj+gXn/S5ahsiTlK3TmC85qgirsdTP/+DeaC4=
github.com/goccy/go-json v0.10.5/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M=
github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=
github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=
github.com/golang/snappy v1.0.0 h1:Oy607GVXHs7RtbggtPBnr2RmDArIsAefDwvrdWvRhGs=
github.com/golang/snappy v1.0.0/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/google/flatbuffers v25.2.10+incompatible h1:F3vclr7C3HpB1k9mxCGRMXq6FdUalZ6H/pNX4FP1v0Q=
github.com/google/flatbuffers v25.2.10+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8=
github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8=
github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU=
github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=
github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/klauspost/asmfmt v1.3.2 h1:4Ri7ox3EwapiOjCki+hw14RyKk201CN4rzyCJRFLpK4=
github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE=
github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo=
github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ=
github.com/klauspost/cpuid/v2 v2.2.11 h1:0OwqZRYI2rFrjS4kvkDnqJkKHdHaRnCm68/DY4OxRzU=
github.com/klauspost/cpuid/v2 v2.2.11/go.mod h1:hqwkgyIinND0mEev00jJYCxPNVRVXFQeu1XKlok6oO0=
github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0=
github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY=
github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI=
github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE=
github.com/pierrec/lz4/v4 v4.1.22 h1:cKFw6uJDK+/gfw5BcDL0JL5aBsAFdsIT18eRtLj7VIU=
github.com/pierrec/lz4/v4 v4.1.22/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4=
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U=
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k=
github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA=
github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY=
github.com/zeebo/assert v1.3.0 h1:g7C04CbJuIDKNPFHmsk4hwZDO5O+kntRxzaUoNXj+IQ=
github.com/zeebo/assert v1.3.0/go.mod h1:Pq9JiuJQpG8JLJdtkwrJESF0Foym2/D9XMU5ciN/wJ0=
github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0=
github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA=
go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA=
go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A=
go.opentelemetry.io/otel v1.37.0 h1:9zhNfelUvx0KBfu/gb+ZgeAfAgtWrfHJZcAqFC228wQ=
go.opentelemetry.io/otel v1.37.0/go.mod h1:ehE/umFRLnuLa/vSccNq9oS1ErUlkkK71gMcN34UG8I=
go.opentelemetry.io/otel/metric v1.37.0 h1:mvwbQS5m0tbmqML4NqK+e3aDiO02vsf/WgbsdpcPoZE=
go.opentelemetry.io/otel/metric v1.37.0/go.mod h1:04wGrZurHYKOc+RKeye86GwKiTb9FKm1WHtO+4EVr2E=
go.opentelemetry.io/otel/sdk v1.37.0 h1:ItB0QUqnjesGRvNcmAcU0LyvkVyGJ2xftD29bWdDvKI=
go.opentelemetry.io/otel/sdk v1.37.0/go.mod h1:VredYzxUvuo2q3WRcDnKDjbdvmO0sCzOvVAiY+yUkAg=
go.opentelemetry.io/otel/sdk/metric v1.37.0 h1:90lI228XrB9jCMuSdA0673aubgRobVZFhbjxHHspCPc=
go.opentelemetry.io/otel/sdk/metric v1.37.0/go.mod h1:cNen4ZWfiD37l5NhS+Keb5RXVWZWpRE+9WyVCpbo5ps=
go.opentelemetry.io/otel/trace v1.37.0 h1:HLdcFNbRQBE2imdSEgm/kwqmQj1Or1l/7bW6mxVK7z4=
go.opentelemetry.io/otel/trace v1.37.0/go.mod h1:TlgrlQ+PtQO5XFerSPUYG0JSgGyryXewPGyayAWSBS0=
golang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0 h1:R84qjqJb5nVJMxqWYb3np9L5ZsaDtB+a39EqjV0JSUM=
golang.org/x/exp v0.0.0-20250408133849-7e4ce0ab07d0/go.mod h1:S9Xr4PYopiDyqSyp5NjCrhFrqg6A5zA2E/iPHPhqnS8=
golang.org/x/mod v0.25.0 h1:n7a+ZbQKQA/Ysbyb0/6IbB1H/X41mKgbhfv7AfG/44w=
golang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww=
golang.org/x/net v0.41.0 h1:vBTly1HeNPEn3wtREYfy4GZ/NECgw2Cnl+nK6Nz3uvw=
golang.org/x/net v0.41.0/go.mod h1:B/K4NNqkfmg07DQYrbwvSluqCJOOXwUjeb/5lOisjbA=
golang.org/x/oauth2 v0.30.0 h1:dnDm7JmhM45NNpd8FDDeLhK6FwqbOf4MLCM9zb1BOHI=
golang.org/x/oauth2 v0.30.0/go.mod h1:B++QgG3ZKulg6sRPGD/mqlHQs5rB3Ml9erfeDY7xKlU=
golang.org/x/sync v0.15.0 h1:KWH3jNZsfyT6xfAfKiz6MRNmd46ByHDYaZ7KSkCtdW8=
golang.org/x/sync v0.15.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA=
golang.org/x/sys v0.33.0 h1:q3i8TbbEz+JRD9ywIRlyRAQbM0qF7hu24q3teo2hbuw=
golang.org/x/sys v0.33.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k=
golang.org/x/text v0.26.0 h1:P42AVeLghgTYr4+xUnTRKDMqpar+PtX7KWuNQL21L8M=
golang.org/x/text v0.26.0/go.mod h1:QK15LZJUUQVJxhz7wXgxSy/CJaTFjd0G+YLonydOVQA=
golang.org/x/tools v0.34.0 h1:qIpSLOxeCYGg9TrcJokLBG4KFA6d795g0xkBkiESGlo=
golang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg=
golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY=
golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90=
gonum.org/v1/gonum v0.16.0 h1:5+ul4Swaf3ESvrOnidPp4GZbzf0mxVQpDCYUQE7OJfk=
gonum.org/v1/gonum v0.16.0/go.mod h1:fef3am4MQ93R2HHpKnLk4/Tbh/s0+wqD5nfa6Pnwy4E=
google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7 h1:pFyd6EwwL2TqFf8emdthzeX+gZE1ElRq3iM8pui4KBY=
google.golang.org/genproto/googleapis/rpc v0.0.0-20250707201910-8d1bb00bc6a7/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A=
google.golang.org/grpc v1.75.0 h1:+TW+dqTd2Biwe6KKfhE5JpiYIBWq865PhKGSXiivqt4=
google.golang.org/grpc v1.75.0/go.mod h1:JtPAzKiq4v1xcAB2hydNlWI2RnF85XXcV0mhKXr2ecQ=
google.golang.org/protobuf v1.36.7 h1:IgrO7UwFQGJdRNXH/sQux4R1Dj1WAKcLElzeeRaXV2A=
google.golang.org/protobuf v1.36.7/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI=
gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
================================================
FILE: internal/generated/base.pb.go
================================================
//
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.30.0
// protoc (unknown)
// source: spark/connect/base.proto
package generated
import (
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
anypb "google.golang.org/protobuf/types/known/anypb"
reflect "reflect"
sync "sync"
)
const (
// Verify that this generated code is sufficiently up-to-date.
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
// Verify that runtime/protoimpl is sufficiently up-to-date.
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
)
// Plan explanation mode.
type AnalyzePlanRequest_Explain_ExplainMode int32
const (
AnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED AnalyzePlanRequest_Explain_ExplainMode = 0
// Generates only physical plan.
AnalyzePlanRequest_Explain_EXPLAIN_MODE_SIMPLE AnalyzePlanRequest_Explain_ExplainMode = 1
// Generates parsed logical plan, analyzed logical plan, optimized logical plan and physical plan.
// Parsed Logical plan is a unresolved plan that extracted from the query. Analyzed logical plans
// transforms which translates unresolvedAttribute and unresolvedRelation into fully typed objects.
// The optimized logical plan transforms through a set of optimization rules, resulting in the
// physical plan.
AnalyzePlanRequest_Explain_EXPLAIN_MODE_EXTENDED AnalyzePlanRequest_Explain_ExplainMode = 2
// Generates code for the statement, if any and a physical plan.
AnalyzePlanRequest_Explain_EXPLAIN_MODE_CODEGEN AnalyzePlanRequest_Explain_ExplainMode = 3
// If plan node statistics are available, generates a logical plan and also the statistics.
AnalyzePlanRequest_Explain_EXPLAIN_MODE_COST AnalyzePlanRequest_Explain_ExplainMode = 4
// Generates a physical plan outline and also node details.
AnalyzePlanRequest_Explain_EXPLAIN_MODE_FORMATTED AnalyzePlanRequest_Explain_ExplainMode = 5
)
// Enum value maps for AnalyzePlanRequest_Explain_ExplainMode.
var (
AnalyzePlanRequest_Explain_ExplainMode_name = map[int32]string{
0: "EXPLAIN_MODE_UNSPECIFIED",
1: "EXPLAIN_MODE_SIMPLE",
2: "EXPLAIN_MODE_EXTENDED",
3: "EXPLAIN_MODE_CODEGEN",
4: "EXPLAIN_MODE_COST",
5: "EXPLAIN_MODE_FORMATTED",
}
AnalyzePlanRequest_Explain_ExplainMode_value = map[string]int32{
"EXPLAIN_MODE_UNSPECIFIED": 0,
"EXPLAIN_MODE_SIMPLE": 1,
"EXPLAIN_MODE_EXTENDED": 2,
"EXPLAIN_MODE_CODEGEN": 3,
"EXPLAIN_MODE_COST": 4,
"EXPLAIN_MODE_FORMATTED": 5,
}
)
func (x AnalyzePlanRequest_Explain_ExplainMode) Enum() *AnalyzePlanRequest_Explain_ExplainMode {
p := new(AnalyzePlanRequest_Explain_ExplainMode)
*p = x
return p
}
func (x AnalyzePlanRequest_Explain_ExplainMode) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (AnalyzePlanRequest_Explain_ExplainMode) Descriptor() protoreflect.EnumDescriptor {
return file_spark_connect_base_proto_enumTypes[0].Descriptor()
}
func (AnalyzePlanRequest_Explain_ExplainMode) Type() protoreflect.EnumType {
return &file_spark_connect_base_proto_enumTypes[0]
}
func (x AnalyzePlanRequest_Explain_ExplainMode) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use AnalyzePlanRequest_Explain_ExplainMode.Descriptor instead.
func (AnalyzePlanRequest_Explain_ExplainMode) EnumDescriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 1, 0}
}
type InterruptRequest_InterruptType int32
const (
InterruptRequest_INTERRUPT_TYPE_UNSPECIFIED InterruptRequest_InterruptType = 0
// Interrupt all running executions within the session with the provided session_id.
InterruptRequest_INTERRUPT_TYPE_ALL InterruptRequest_InterruptType = 1
// Interrupt all running executions within the session with the provided operation_tag.
InterruptRequest_INTERRUPT_TYPE_TAG InterruptRequest_InterruptType = 2
// Interrupt the running execution within the session with the provided operation_id.
InterruptRequest_INTERRUPT_TYPE_OPERATION_ID InterruptRequest_InterruptType = 3
)
// Enum value maps for InterruptRequest_InterruptType.
var (
InterruptRequest_InterruptType_name = map[int32]string{
0: "INTERRUPT_TYPE_UNSPECIFIED",
1: "INTERRUPT_TYPE_ALL",
2: "INTERRUPT_TYPE_TAG",
3: "INTERRUPT_TYPE_OPERATION_ID",
}
InterruptRequest_InterruptType_value = map[string]int32{
"INTERRUPT_TYPE_UNSPECIFIED": 0,
"INTERRUPT_TYPE_ALL": 1,
"INTERRUPT_TYPE_TAG": 2,
"INTERRUPT_TYPE_OPERATION_ID": 3,
}
)
func (x InterruptRequest_InterruptType) Enum() *InterruptRequest_InterruptType {
p := new(InterruptRequest_InterruptType)
*p = x
return p
}
func (x InterruptRequest_InterruptType) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (InterruptRequest_InterruptType) Descriptor() protoreflect.EnumDescriptor {
return file_spark_connect_base_proto_enumTypes[1].Descriptor()
}
func (InterruptRequest_InterruptType) Type() protoreflect.EnumType {
return &file_spark_connect_base_proto_enumTypes[1]
}
func (x InterruptRequest_InterruptType) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use InterruptRequest_InterruptType.Descriptor instead.
func (InterruptRequest_InterruptType) EnumDescriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{13, 0}
}
// The type of this query context.
type FetchErrorDetailsResponse_QueryContext_ContextType int32
const (
FetchErrorDetailsResponse_QueryContext_SQL FetchErrorDetailsResponse_QueryContext_ContextType = 0
FetchErrorDetailsResponse_QueryContext_DATAFRAME FetchErrorDetailsResponse_QueryContext_ContextType = 1
)
// Enum value maps for FetchErrorDetailsResponse_QueryContext_ContextType.
var (
FetchErrorDetailsResponse_QueryContext_ContextType_name = map[int32]string{
0: "SQL",
1: "DATAFRAME",
}
FetchErrorDetailsResponse_QueryContext_ContextType_value = map[string]int32{
"SQL": 0,
"DATAFRAME": 1,
}
)
func (x FetchErrorDetailsResponse_QueryContext_ContextType) Enum() *FetchErrorDetailsResponse_QueryContext_ContextType {
p := new(FetchErrorDetailsResponse_QueryContext_ContextType)
*p = x
return p
}
func (x FetchErrorDetailsResponse_QueryContext_ContextType) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (FetchErrorDetailsResponse_QueryContext_ContextType) Descriptor() protoreflect.EnumDescriptor {
return file_spark_connect_base_proto_enumTypes[2].Descriptor()
}
func (FetchErrorDetailsResponse_QueryContext_ContextType) Type() protoreflect.EnumType {
return &file_spark_connect_base_proto_enumTypes[2]
}
func (x FetchErrorDetailsResponse_QueryContext_ContextType) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use FetchErrorDetailsResponse_QueryContext_ContextType.Descriptor instead.
func (FetchErrorDetailsResponse_QueryContext_ContextType) EnumDescriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{22, 1, 0}
}
// A [[Plan]] is the structure that carries the runtime information for the execution from the
// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference
// to the underlying logical plan or it can be of the [[Command]] type that is used to execute
// commands on the server.
type Plan struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Types that are assignable to OpType:
//
// *Plan_Root
// *Plan_Command
OpType isPlan_OpType `protobuf_oneof:"op_type"`
}
func (x *Plan) Reset() {
*x = Plan{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[0]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *Plan) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*Plan) ProtoMessage() {}
func (x *Plan) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[0]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use Plan.ProtoReflect.Descriptor instead.
func (*Plan) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{0}
}
func (m *Plan) GetOpType() isPlan_OpType {
if m != nil {
return m.OpType
}
return nil
}
func (x *Plan) GetRoot() *Relation {
if x, ok := x.GetOpType().(*Plan_Root); ok {
return x.Root
}
return nil
}
func (x *Plan) GetCommand() *Command {
if x, ok := x.GetOpType().(*Plan_Command); ok {
return x.Command
}
return nil
}
type isPlan_OpType interface {
isPlan_OpType()
}
type Plan_Root struct {
Root *Relation `protobuf:"bytes,1,opt,name=root,proto3,oneof"`
}
type Plan_Command struct {
Command *Command `protobuf:"bytes,2,opt,name=command,proto3,oneof"`
}
func (*Plan_Root) isPlan_OpType() {}
func (*Plan_Command) isPlan_OpType() {}
// User Context is used to refer to one particular user session that is executing
// queries in the backend.
type UserContext struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
UserId string `protobuf:"bytes,1,opt,name=user_id,json=userId,proto3" json:"user_id,omitempty"`
UserName string `protobuf:"bytes,2,opt,name=user_name,json=userName,proto3" json:"user_name,omitempty"`
// To extend the existing user context message that is used to identify incoming requests,
// Spark Connect leverages the Any protobuf type that can be used to inject arbitrary other
// messages into this message. Extensions are stored as a `repeated` type to be able to
// handle multiple active extensions.
Extensions []*anypb.Any `protobuf:"bytes,999,rep,name=extensions,proto3" json:"extensions,omitempty"`
}
func (x *UserContext) Reset() {
*x = UserContext{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[1]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *UserContext) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*UserContext) ProtoMessage() {}
func (x *UserContext) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[1]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use UserContext.ProtoReflect.Descriptor instead.
func (*UserContext) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{1}
}
func (x *UserContext) GetUserId() string {
if x != nil {
return x.UserId
}
return ""
}
func (x *UserContext) GetUserName() string {
if x != nil {
return x.UserName
}
return ""
}
func (x *UserContext) GetExtensions() []*anypb.Any {
if x != nil {
return x.Extensions
}
return nil
}
// Request to perform plan analyze, optionally to explain the plan.
type AnalyzePlanRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,17,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,3,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// Types that are assignable to Analyze:
//
// *AnalyzePlanRequest_Schema_
// *AnalyzePlanRequest_Explain_
// *AnalyzePlanRequest_TreeString_
// *AnalyzePlanRequest_IsLocal_
// *AnalyzePlanRequest_IsStreaming_
// *AnalyzePlanRequest_InputFiles_
// *AnalyzePlanRequest_SparkVersion_
// *AnalyzePlanRequest_DdlParse
// *AnalyzePlanRequest_SameSemantics_
// *AnalyzePlanRequest_SemanticHash_
// *AnalyzePlanRequest_Persist_
// *AnalyzePlanRequest_Unpersist_
// *AnalyzePlanRequest_GetStorageLevel_
// *AnalyzePlanRequest_JsonToDdl
Analyze isAnalyzePlanRequest_Analyze `protobuf_oneof:"analyze"`
}
func (x *AnalyzePlanRequest) Reset() {
*x = AnalyzePlanRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest) ProtoMessage() {}
func (x *AnalyzePlanRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[2]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2}
}
func (x *AnalyzePlanRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *AnalyzePlanRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *AnalyzePlanRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *AnalyzePlanRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (m *AnalyzePlanRequest) GetAnalyze() isAnalyzePlanRequest_Analyze {
if m != nil {
return m.Analyze
}
return nil
}
func (x *AnalyzePlanRequest) GetSchema() *AnalyzePlanRequest_Schema {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Schema_); ok {
return x.Schema
}
return nil
}
func (x *AnalyzePlanRequest) GetExplain() *AnalyzePlanRequest_Explain {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Explain_); ok {
return x.Explain
}
return nil
}
func (x *AnalyzePlanRequest) GetTreeString() *AnalyzePlanRequest_TreeString {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_TreeString_); ok {
return x.TreeString
}
return nil
}
func (x *AnalyzePlanRequest) GetIsLocal() *AnalyzePlanRequest_IsLocal {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_IsLocal_); ok {
return x.IsLocal
}
return nil
}
func (x *AnalyzePlanRequest) GetIsStreaming() *AnalyzePlanRequest_IsStreaming {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_IsStreaming_); ok {
return x.IsStreaming
}
return nil
}
func (x *AnalyzePlanRequest) GetInputFiles() *AnalyzePlanRequest_InputFiles {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_InputFiles_); ok {
return x.InputFiles
}
return nil
}
func (x *AnalyzePlanRequest) GetSparkVersion() *AnalyzePlanRequest_SparkVersion {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SparkVersion_); ok {
return x.SparkVersion
}
return nil
}
func (x *AnalyzePlanRequest) GetDdlParse() *AnalyzePlanRequest_DDLParse {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_DdlParse); ok {
return x.DdlParse
}
return nil
}
func (x *AnalyzePlanRequest) GetSameSemantics() *AnalyzePlanRequest_SameSemantics {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SameSemantics_); ok {
return x.SameSemantics
}
return nil
}
func (x *AnalyzePlanRequest) GetSemanticHash() *AnalyzePlanRequest_SemanticHash {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_SemanticHash_); ok {
return x.SemanticHash
}
return nil
}
func (x *AnalyzePlanRequest) GetPersist() *AnalyzePlanRequest_Persist {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Persist_); ok {
return x.Persist
}
return nil
}
func (x *AnalyzePlanRequest) GetUnpersist() *AnalyzePlanRequest_Unpersist {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_Unpersist_); ok {
return x.Unpersist
}
return nil
}
func (x *AnalyzePlanRequest) GetGetStorageLevel() *AnalyzePlanRequest_GetStorageLevel {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_GetStorageLevel_); ok {
return x.GetStorageLevel
}
return nil
}
func (x *AnalyzePlanRequest) GetJsonToDdl() *AnalyzePlanRequest_JsonToDDL {
if x, ok := x.GetAnalyze().(*AnalyzePlanRequest_JsonToDdl); ok {
return x.JsonToDdl
}
return nil
}
type isAnalyzePlanRequest_Analyze interface {
isAnalyzePlanRequest_Analyze()
}
type AnalyzePlanRequest_Schema_ struct {
Schema *AnalyzePlanRequest_Schema `protobuf:"bytes,4,opt,name=schema,proto3,oneof"`
}
type AnalyzePlanRequest_Explain_ struct {
Explain *AnalyzePlanRequest_Explain `protobuf:"bytes,5,opt,name=explain,proto3,oneof"`
}
type AnalyzePlanRequest_TreeString_ struct {
TreeString *AnalyzePlanRequest_TreeString `protobuf:"bytes,6,opt,name=tree_string,json=treeString,proto3,oneof"`
}
type AnalyzePlanRequest_IsLocal_ struct {
IsLocal *AnalyzePlanRequest_IsLocal `protobuf:"bytes,7,opt,name=is_local,json=isLocal,proto3,oneof"`
}
type AnalyzePlanRequest_IsStreaming_ struct {
IsStreaming *AnalyzePlanRequest_IsStreaming `protobuf:"bytes,8,opt,name=is_streaming,json=isStreaming,proto3,oneof"`
}
type AnalyzePlanRequest_InputFiles_ struct {
InputFiles *AnalyzePlanRequest_InputFiles `protobuf:"bytes,9,opt,name=input_files,json=inputFiles,proto3,oneof"`
}
type AnalyzePlanRequest_SparkVersion_ struct {
SparkVersion *AnalyzePlanRequest_SparkVersion `protobuf:"bytes,10,opt,name=spark_version,json=sparkVersion,proto3,oneof"`
}
type AnalyzePlanRequest_DdlParse struct {
DdlParse *AnalyzePlanRequest_DDLParse `protobuf:"bytes,11,opt,name=ddl_parse,json=ddlParse,proto3,oneof"`
}
type AnalyzePlanRequest_SameSemantics_ struct {
SameSemantics *AnalyzePlanRequest_SameSemantics `protobuf:"bytes,12,opt,name=same_semantics,json=sameSemantics,proto3,oneof"`
}
type AnalyzePlanRequest_SemanticHash_ struct {
SemanticHash *AnalyzePlanRequest_SemanticHash `protobuf:"bytes,13,opt,name=semantic_hash,json=semanticHash,proto3,oneof"`
}
type AnalyzePlanRequest_Persist_ struct {
Persist *AnalyzePlanRequest_Persist `protobuf:"bytes,14,opt,name=persist,proto3,oneof"`
}
type AnalyzePlanRequest_Unpersist_ struct {
Unpersist *AnalyzePlanRequest_Unpersist `protobuf:"bytes,15,opt,name=unpersist,proto3,oneof"`
}
type AnalyzePlanRequest_GetStorageLevel_ struct {
GetStorageLevel *AnalyzePlanRequest_GetStorageLevel `protobuf:"bytes,16,opt,name=get_storage_level,json=getStorageLevel,proto3,oneof"`
}
type AnalyzePlanRequest_JsonToDdl struct {
JsonToDdl *AnalyzePlanRequest_JsonToDDL `protobuf:"bytes,18,opt,name=json_to_ddl,json=jsonToDdl,proto3,oneof"`
}
func (*AnalyzePlanRequest_Schema_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_Explain_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_TreeString_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_IsLocal_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_IsStreaming_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_InputFiles_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_SparkVersion_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_DdlParse) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_SameSemantics_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_SemanticHash_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_Persist_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_Unpersist_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_GetStorageLevel_) isAnalyzePlanRequest_Analyze() {}
func (*AnalyzePlanRequest_JsonToDdl) isAnalyzePlanRequest_Analyze() {}
// Response to performing analysis of the query. Contains relevant metadata to be able to
// reason about the performance.
// Next ID: 16
type AnalyzePlanResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,15,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// Types that are assignable to Result:
//
// *AnalyzePlanResponse_Schema_
// *AnalyzePlanResponse_Explain_
// *AnalyzePlanResponse_TreeString_
// *AnalyzePlanResponse_IsLocal_
// *AnalyzePlanResponse_IsStreaming_
// *AnalyzePlanResponse_InputFiles_
// *AnalyzePlanResponse_SparkVersion_
// *AnalyzePlanResponse_DdlParse
// *AnalyzePlanResponse_SameSemantics_
// *AnalyzePlanResponse_SemanticHash_
// *AnalyzePlanResponse_Persist_
// *AnalyzePlanResponse_Unpersist_
// *AnalyzePlanResponse_GetStorageLevel_
// *AnalyzePlanResponse_JsonToDdl
Result isAnalyzePlanResponse_Result `protobuf_oneof:"result"`
}
func (x *AnalyzePlanResponse) Reset() {
*x = AnalyzePlanResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[3]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanResponse) ProtoMessage() {}
func (x *AnalyzePlanResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[3]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanResponse.ProtoReflect.Descriptor instead.
func (*AnalyzePlanResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{3}
}
func (x *AnalyzePlanResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *AnalyzePlanResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (m *AnalyzePlanResponse) GetResult() isAnalyzePlanResponse_Result {
if m != nil {
return m.Result
}
return nil
}
func (x *AnalyzePlanResponse) GetSchema() *AnalyzePlanResponse_Schema {
if x, ok := x.GetResult().(*AnalyzePlanResponse_Schema_); ok {
return x.Schema
}
return nil
}
func (x *AnalyzePlanResponse) GetExplain() *AnalyzePlanResponse_Explain {
if x, ok := x.GetResult().(*AnalyzePlanResponse_Explain_); ok {
return x.Explain
}
return nil
}
func (x *AnalyzePlanResponse) GetTreeString() *AnalyzePlanResponse_TreeString {
if x, ok := x.GetResult().(*AnalyzePlanResponse_TreeString_); ok {
return x.TreeString
}
return nil
}
func (x *AnalyzePlanResponse) GetIsLocal() *AnalyzePlanResponse_IsLocal {
if x, ok := x.GetResult().(*AnalyzePlanResponse_IsLocal_); ok {
return x.IsLocal
}
return nil
}
func (x *AnalyzePlanResponse) GetIsStreaming() *AnalyzePlanResponse_IsStreaming {
if x, ok := x.GetResult().(*AnalyzePlanResponse_IsStreaming_); ok {
return x.IsStreaming
}
return nil
}
func (x *AnalyzePlanResponse) GetInputFiles() *AnalyzePlanResponse_InputFiles {
if x, ok := x.GetResult().(*AnalyzePlanResponse_InputFiles_); ok {
return x.InputFiles
}
return nil
}
func (x *AnalyzePlanResponse) GetSparkVersion() *AnalyzePlanResponse_SparkVersion {
if x, ok := x.GetResult().(*AnalyzePlanResponse_SparkVersion_); ok {
return x.SparkVersion
}
return nil
}
func (x *AnalyzePlanResponse) GetDdlParse() *AnalyzePlanResponse_DDLParse {
if x, ok := x.GetResult().(*AnalyzePlanResponse_DdlParse); ok {
return x.DdlParse
}
return nil
}
func (x *AnalyzePlanResponse) GetSameSemantics() *AnalyzePlanResponse_SameSemantics {
if x, ok := x.GetResult().(*AnalyzePlanResponse_SameSemantics_); ok {
return x.SameSemantics
}
return nil
}
func (x *AnalyzePlanResponse) GetSemanticHash() *AnalyzePlanResponse_SemanticHash {
if x, ok := x.GetResult().(*AnalyzePlanResponse_SemanticHash_); ok {
return x.SemanticHash
}
return nil
}
func (x *AnalyzePlanResponse) GetPersist() *AnalyzePlanResponse_Persist {
if x, ok := x.GetResult().(*AnalyzePlanResponse_Persist_); ok {
return x.Persist
}
return nil
}
func (x *AnalyzePlanResponse) GetUnpersist() *AnalyzePlanResponse_Unpersist {
if x, ok := x.GetResult().(*AnalyzePlanResponse_Unpersist_); ok {
return x.Unpersist
}
return nil
}
func (x *AnalyzePlanResponse) GetGetStorageLevel() *AnalyzePlanResponse_GetStorageLevel {
if x, ok := x.GetResult().(*AnalyzePlanResponse_GetStorageLevel_); ok {
return x.GetStorageLevel
}
return nil
}
func (x *AnalyzePlanResponse) GetJsonToDdl() *AnalyzePlanResponse_JsonToDDL {
if x, ok := x.GetResult().(*AnalyzePlanResponse_JsonToDdl); ok {
return x.JsonToDdl
}
return nil
}
type isAnalyzePlanResponse_Result interface {
isAnalyzePlanResponse_Result()
}
type AnalyzePlanResponse_Schema_ struct {
Schema *AnalyzePlanResponse_Schema `protobuf:"bytes,2,opt,name=schema,proto3,oneof"`
}
type AnalyzePlanResponse_Explain_ struct {
Explain *AnalyzePlanResponse_Explain `protobuf:"bytes,3,opt,name=explain,proto3,oneof"`
}
type AnalyzePlanResponse_TreeString_ struct {
TreeString *AnalyzePlanResponse_TreeString `protobuf:"bytes,4,opt,name=tree_string,json=treeString,proto3,oneof"`
}
type AnalyzePlanResponse_IsLocal_ struct {
IsLocal *AnalyzePlanResponse_IsLocal `protobuf:"bytes,5,opt,name=is_local,json=isLocal,proto3,oneof"`
}
type AnalyzePlanResponse_IsStreaming_ struct {
IsStreaming *AnalyzePlanResponse_IsStreaming `protobuf:"bytes,6,opt,name=is_streaming,json=isStreaming,proto3,oneof"`
}
type AnalyzePlanResponse_InputFiles_ struct {
InputFiles *AnalyzePlanResponse_InputFiles `protobuf:"bytes,7,opt,name=input_files,json=inputFiles,proto3,oneof"`
}
type AnalyzePlanResponse_SparkVersion_ struct {
SparkVersion *AnalyzePlanResponse_SparkVersion `protobuf:"bytes,8,opt,name=spark_version,json=sparkVersion,proto3,oneof"`
}
type AnalyzePlanResponse_DdlParse struct {
DdlParse *AnalyzePlanResponse_DDLParse `protobuf:"bytes,9,opt,name=ddl_parse,json=ddlParse,proto3,oneof"`
}
type AnalyzePlanResponse_SameSemantics_ struct {
SameSemantics *AnalyzePlanResponse_SameSemantics `protobuf:"bytes,10,opt,name=same_semantics,json=sameSemantics,proto3,oneof"`
}
type AnalyzePlanResponse_SemanticHash_ struct {
SemanticHash *AnalyzePlanResponse_SemanticHash `protobuf:"bytes,11,opt,name=semantic_hash,json=semanticHash,proto3,oneof"`
}
type AnalyzePlanResponse_Persist_ struct {
Persist *AnalyzePlanResponse_Persist `protobuf:"bytes,12,opt,name=persist,proto3,oneof"`
}
type AnalyzePlanResponse_Unpersist_ struct {
Unpersist *AnalyzePlanResponse_Unpersist `protobuf:"bytes,13,opt,name=unpersist,proto3,oneof"`
}
type AnalyzePlanResponse_GetStorageLevel_ struct {
GetStorageLevel *AnalyzePlanResponse_GetStorageLevel `protobuf:"bytes,14,opt,name=get_storage_level,json=getStorageLevel,proto3,oneof"`
}
type AnalyzePlanResponse_JsonToDdl struct {
JsonToDdl *AnalyzePlanResponse_JsonToDDL `protobuf:"bytes,16,opt,name=json_to_ddl,json=jsonToDdl,proto3,oneof"`
}
func (*AnalyzePlanResponse_Schema_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_Explain_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_TreeString_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_IsLocal_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_IsStreaming_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_InputFiles_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_SparkVersion_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_DdlParse) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_SameSemantics_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_SemanticHash_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_Persist_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_Unpersist_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_GetStorageLevel_) isAnalyzePlanResponse_Result() {}
func (*AnalyzePlanResponse_JsonToDdl) isAnalyzePlanResponse_Result() {}
// A request to be executed by the service.
type ExecutePlanRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,8,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
//
// user_context.user_id and session+id both identify a unique remote spark session on the
// server side.
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Optional)
// Provide an id for this request. If not provided, it will be generated by the server.
// It is returned in every ExecutePlanResponse.operation_id of the ExecutePlan response stream.
// The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
OperationId *string `protobuf:"bytes,6,opt,name=operation_id,json=operationId,proto3,oneof" json:"operation_id,omitempty"`
// (Required) The logical plan to be executed / analyzed.
Plan *Plan `protobuf:"bytes,3,opt,name=plan,proto3" json:"plan,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,4,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// Repeated element for options that can be passed to the request. This element is currently
// unused but allows to pass in an extension value used for arbitrary options.
RequestOptions []*ExecutePlanRequest_RequestOption `protobuf:"bytes,5,rep,name=request_options,json=requestOptions,proto3" json:"request_options,omitempty"`
// Tags to tag the given execution with.
// Tags cannot contain ',' character and cannot be empty strings.
// Used by Interrupt with interrupt.tag.
Tags []string `protobuf:"bytes,7,rep,name=tags,proto3" json:"tags,omitempty"`
}
func (x *ExecutePlanRequest) Reset() {
*x = ExecutePlanRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[4]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ExecutePlanRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExecutePlanRequest) ProtoMessage() {}
func (x *ExecutePlanRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[4]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ExecutePlanRequest.ProtoReflect.Descriptor instead.
func (*ExecutePlanRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{4}
}
func (x *ExecutePlanRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ExecutePlanRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *ExecutePlanRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ExecutePlanRequest) GetOperationId() string {
if x != nil && x.OperationId != nil {
return *x.OperationId
}
return ""
}
func (x *ExecutePlanRequest) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
func (x *ExecutePlanRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (x *ExecutePlanRequest) GetRequestOptions() []*ExecutePlanRequest_RequestOption {
if x != nil {
return x.RequestOptions
}
return nil
}
func (x *ExecutePlanRequest) GetTags() []string {
if x != nil {
return x.Tags
}
return nil
}
// The response of a query, can be one or more for each request. Responses belonging to the
// same input query, carry the same `session_id`.
// Next ID: 17
type ExecutePlanResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,15,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// Identifies the ExecutePlan execution.
// If set by the client in ExecutePlanRequest.operationId, that value is returned.
// Otherwise generated by the server.
// It is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
OperationId string `protobuf:"bytes,12,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"`
// Identified the response in the stream.
// The id is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
ResponseId string `protobuf:"bytes,13,opt,name=response_id,json=responseId,proto3" json:"response_id,omitempty"`
// Union type for the different response messages.
//
// Types that are assignable to ResponseType:
//
// *ExecutePlanResponse_ArrowBatch_
// *ExecutePlanResponse_SqlCommandResult_
// *ExecutePlanResponse_WriteStreamOperationStartResult
// *ExecutePlanResponse_StreamingQueryCommandResult
// *ExecutePlanResponse_GetResourcesCommandResult
// *ExecutePlanResponse_StreamingQueryManagerCommandResult
// *ExecutePlanResponse_StreamingQueryListenerEventsResult
// *ExecutePlanResponse_ResultComplete_
// *ExecutePlanResponse_CreateResourceProfileCommandResult
// *ExecutePlanResponse_ExecutionProgress_
// *ExecutePlanResponse_CheckpointCommandResult
// *ExecutePlanResponse_MlCommandResult
// *ExecutePlanResponse_PipelineEventResult
// *ExecutePlanResponse_PipelineCommandResult
// *ExecutePlanResponse_Extension
ResponseType isExecutePlanResponse_ResponseType `protobuf_oneof:"response_type"`
// Metrics for the query execution. Typically, this field is only present in the last
// batch of results and then represent the overall state of the query execution.
Metrics *ExecutePlanResponse_Metrics `protobuf:"bytes,4,opt,name=metrics,proto3" json:"metrics,omitempty"`
// The metrics observed during the execution of the query plan.
ObservedMetrics []*ExecutePlanResponse_ObservedMetrics `protobuf:"bytes,6,rep,name=observed_metrics,json=observedMetrics,proto3" json:"observed_metrics,omitempty"`
// (Optional) The Spark schema. This field is available when `collect` is called.
Schema *DataType `protobuf:"bytes,7,opt,name=schema,proto3" json:"schema,omitempty"`
}
func (x *ExecutePlanResponse) Reset() {
*x = ExecutePlanResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[5]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ExecutePlanResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExecutePlanResponse) ProtoMessage() {}
func (x *ExecutePlanResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[5]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ExecutePlanResponse.ProtoReflect.Descriptor instead.
func (*ExecutePlanResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{5}
}
func (x *ExecutePlanResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ExecutePlanResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *ExecutePlanResponse) GetOperationId() string {
if x != nil {
return x.OperationId
}
return ""
}
func (x *ExecutePlanResponse) GetResponseId() string {
if x != nil {
return x.ResponseId
}
return ""
}
func (m *ExecutePlanResponse) GetResponseType() isExecutePlanResponse_ResponseType {
if m != nil {
return m.ResponseType
}
return nil
}
func (x *ExecutePlanResponse) GetArrowBatch() *ExecutePlanResponse_ArrowBatch {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_ArrowBatch_); ok {
return x.ArrowBatch
}
return nil
}
func (x *ExecutePlanResponse) GetSqlCommandResult() *ExecutePlanResponse_SqlCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_SqlCommandResult_); ok {
return x.SqlCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetWriteStreamOperationStartResult() *WriteStreamOperationStartResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_WriteStreamOperationStartResult); ok {
return x.WriteStreamOperationStartResult
}
return nil
}
func (x *ExecutePlanResponse) GetStreamingQueryCommandResult() *StreamingQueryCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryCommandResult); ok {
return x.StreamingQueryCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetGetResourcesCommandResult() *GetResourcesCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_GetResourcesCommandResult); ok {
return x.GetResourcesCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetStreamingQueryManagerCommandResult() *StreamingQueryManagerCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryManagerCommandResult); ok {
return x.StreamingQueryManagerCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetStreamingQueryListenerEventsResult() *StreamingQueryListenerEventsResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_StreamingQueryListenerEventsResult); ok {
return x.StreamingQueryListenerEventsResult
}
return nil
}
func (x *ExecutePlanResponse) GetResultComplete() *ExecutePlanResponse_ResultComplete {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_ResultComplete_); ok {
return x.ResultComplete
}
return nil
}
func (x *ExecutePlanResponse) GetCreateResourceProfileCommandResult() *CreateResourceProfileCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_CreateResourceProfileCommandResult); ok {
return x.CreateResourceProfileCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetExecutionProgress() *ExecutePlanResponse_ExecutionProgress {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_ExecutionProgress_); ok {
return x.ExecutionProgress
}
return nil
}
func (x *ExecutePlanResponse) GetCheckpointCommandResult() *CheckpointCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_CheckpointCommandResult); ok {
return x.CheckpointCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetMlCommandResult() *MlCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_MlCommandResult); ok {
return x.MlCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetPipelineEventResult() *PipelineEventResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_PipelineEventResult); ok {
return x.PipelineEventResult
}
return nil
}
func (x *ExecutePlanResponse) GetPipelineCommandResult() *PipelineCommandResult {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_PipelineCommandResult); ok {
return x.PipelineCommandResult
}
return nil
}
func (x *ExecutePlanResponse) GetExtension() *anypb.Any {
if x, ok := x.GetResponseType().(*ExecutePlanResponse_Extension); ok {
return x.Extension
}
return nil
}
func (x *ExecutePlanResponse) GetMetrics() *ExecutePlanResponse_Metrics {
if x != nil {
return x.Metrics
}
return nil
}
func (x *ExecutePlanResponse) GetObservedMetrics() []*ExecutePlanResponse_ObservedMetrics {
if x != nil {
return x.ObservedMetrics
}
return nil
}
func (x *ExecutePlanResponse) GetSchema() *DataType {
if x != nil {
return x.Schema
}
return nil
}
type isExecutePlanResponse_ResponseType interface {
isExecutePlanResponse_ResponseType()
}
type ExecutePlanResponse_ArrowBatch_ struct {
ArrowBatch *ExecutePlanResponse_ArrowBatch `protobuf:"bytes,2,opt,name=arrow_batch,json=arrowBatch,proto3,oneof"`
}
type ExecutePlanResponse_SqlCommandResult_ struct {
// Special case for executing SQL commands.
SqlCommandResult *ExecutePlanResponse_SqlCommandResult `protobuf:"bytes,5,opt,name=sql_command_result,json=sqlCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_WriteStreamOperationStartResult struct {
// Response for a streaming query.
WriteStreamOperationStartResult *WriteStreamOperationStartResult `protobuf:"bytes,8,opt,name=write_stream_operation_start_result,json=writeStreamOperationStartResult,proto3,oneof"`
}
type ExecutePlanResponse_StreamingQueryCommandResult struct {
// Response for commands on a streaming query.
StreamingQueryCommandResult *StreamingQueryCommandResult `protobuf:"bytes,9,opt,name=streaming_query_command_result,json=streamingQueryCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_GetResourcesCommandResult struct {
// Response for 'SparkContext.resources'.
GetResourcesCommandResult *GetResourcesCommandResult `protobuf:"bytes,10,opt,name=get_resources_command_result,json=getResourcesCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_StreamingQueryManagerCommandResult struct {
// Response for commands on the streaming query manager.
StreamingQueryManagerCommandResult *StreamingQueryManagerCommandResult `protobuf:"bytes,11,opt,name=streaming_query_manager_command_result,json=streamingQueryManagerCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_StreamingQueryListenerEventsResult struct {
// Response for commands on the client side streaming query listener.
StreamingQueryListenerEventsResult *StreamingQueryListenerEventsResult `protobuf:"bytes,16,opt,name=streaming_query_listener_events_result,json=streamingQueryListenerEventsResult,proto3,oneof"`
}
type ExecutePlanResponse_ResultComplete_ struct {
// Response type informing if the stream is complete in reattachable execution.
ResultComplete *ExecutePlanResponse_ResultComplete `protobuf:"bytes,14,opt,name=result_complete,json=resultComplete,proto3,oneof"`
}
type ExecutePlanResponse_CreateResourceProfileCommandResult struct {
// Response for command that creates ResourceProfile.
CreateResourceProfileCommandResult *CreateResourceProfileCommandResult `protobuf:"bytes,17,opt,name=create_resource_profile_command_result,json=createResourceProfileCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_ExecutionProgress_ struct {
// (Optional) Intermediate query progress reports.
ExecutionProgress *ExecutePlanResponse_ExecutionProgress `protobuf:"bytes,18,opt,name=execution_progress,json=executionProgress,proto3,oneof"`
}
type ExecutePlanResponse_CheckpointCommandResult struct {
// Response for command that checkpoints a DataFrame.
CheckpointCommandResult *CheckpointCommandResult `protobuf:"bytes,19,opt,name=checkpoint_command_result,json=checkpointCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_MlCommandResult struct {
// ML command response
MlCommandResult *MlCommandResult `protobuf:"bytes,20,opt,name=ml_command_result,json=mlCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_PipelineEventResult struct {
// Response containing pipeline event that is streamed back to the client during a pipeline run
PipelineEventResult *PipelineEventResult `protobuf:"bytes,21,opt,name=pipeline_event_result,json=pipelineEventResult,proto3,oneof"`
}
type ExecutePlanResponse_PipelineCommandResult struct {
// Pipeline command response
PipelineCommandResult *PipelineCommandResult `protobuf:"bytes,22,opt,name=pipeline_command_result,json=pipelineCommandResult,proto3,oneof"`
}
type ExecutePlanResponse_Extension struct {
// Support arbitrary result objects.
Extension *anypb.Any `protobuf:"bytes,999,opt,name=extension,proto3,oneof"`
}
func (*ExecutePlanResponse_ArrowBatch_) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_SqlCommandResult_) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_WriteStreamOperationStartResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_StreamingQueryCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_GetResourcesCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_StreamingQueryManagerCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_StreamingQueryListenerEventsResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_ResultComplete_) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_CreateResourceProfileCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_ExecutionProgress_) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_CheckpointCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_MlCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_PipelineEventResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_PipelineCommandResult) isExecutePlanResponse_ResponseType() {}
func (*ExecutePlanResponse_Extension) isExecutePlanResponse_ResponseType() {}
// The key-value pair for the config request and response.
type KeyValue struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The key.
Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
// (Optional) The value.
Value *string `protobuf:"bytes,2,opt,name=value,proto3,oneof" json:"value,omitempty"`
}
func (x *KeyValue) Reset() {
*x = KeyValue{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[6]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *KeyValue) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*KeyValue) ProtoMessage() {}
func (x *KeyValue) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[6]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use KeyValue.ProtoReflect.Descriptor instead.
func (*KeyValue) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{6}
}
func (x *KeyValue) GetKey() string {
if x != nil {
return x.Key
}
return ""
}
func (x *KeyValue) GetValue() string {
if x != nil && x.Value != nil {
return *x.Value
}
return ""
}
// Request to update or fetch the configurations.
type ConfigRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,8,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Required) The operation for the config.
Operation *ConfigRequest_Operation `protobuf:"bytes,3,opt,name=operation,proto3" json:"operation,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,4,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
}
func (x *ConfigRequest) Reset() {
*x = ConfigRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[7]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ConfigRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ConfigRequest) ProtoMessage() {}
func (x *ConfigRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[7]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ConfigRequest.ProtoReflect.Descriptor instead.
func (*ConfigRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{7}
}
func (x *ConfigRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ConfigRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *ConfigRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ConfigRequest) GetOperation() *ConfigRequest_Operation {
if x != nil {
return x.Operation
}
return nil
}
func (x *ConfigRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
// Response to the config request.
// Next ID: 5
type ConfigResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,4,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// (Optional) The result key-value pairs.
//
// Available when the operation is 'Get', 'GetWithDefault', 'GetOption', 'GetAll'.
// Also available for the operation 'IsModifiable' with boolean string "true" and "false".
Pairs []*KeyValue `protobuf:"bytes,2,rep,name=pairs,proto3" json:"pairs,omitempty"`
// (Optional)
//
// Warning messages for deprecated or unsupported configurations.
Warnings []string `protobuf:"bytes,3,rep,name=warnings,proto3" json:"warnings,omitempty"`
}
func (x *ConfigResponse) Reset() {
*x = ConfigResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[8]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ConfigResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ConfigResponse) ProtoMessage() {}
func (x *ConfigResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[8]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ConfigResponse.ProtoReflect.Descriptor instead.
func (*ConfigResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{8}
}
func (x *ConfigResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ConfigResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *ConfigResponse) GetPairs() []*KeyValue {
if x != nil {
return x.Pairs
}
return nil
}
func (x *ConfigResponse) GetWarnings() []string {
if x != nil {
return x.Warnings
}
return nil
}
// Request to transfer client-local artifacts.
type AddArtifactsRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,6,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// The payload is either a batch of artifacts or a partial chunk of a large artifact.
//
// Types that are assignable to Payload:
//
// *AddArtifactsRequest_Batch_
// *AddArtifactsRequest_BeginChunk
// *AddArtifactsRequest_Chunk
Payload isAddArtifactsRequest_Payload `protobuf_oneof:"payload"`
}
func (x *AddArtifactsRequest) Reset() {
*x = AddArtifactsRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AddArtifactsRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AddArtifactsRequest) ProtoMessage() {}
func (x *AddArtifactsRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[9]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AddArtifactsRequest.ProtoReflect.Descriptor instead.
func (*AddArtifactsRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{9}
}
func (x *AddArtifactsRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *AddArtifactsRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *AddArtifactsRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *AddArtifactsRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (m *AddArtifactsRequest) GetPayload() isAddArtifactsRequest_Payload {
if m != nil {
return m.Payload
}
return nil
}
func (x *AddArtifactsRequest) GetBatch() *AddArtifactsRequest_Batch {
if x, ok := x.GetPayload().(*AddArtifactsRequest_Batch_); ok {
return x.Batch
}
return nil
}
func (x *AddArtifactsRequest) GetBeginChunk() *AddArtifactsRequest_BeginChunkedArtifact {
if x, ok := x.GetPayload().(*AddArtifactsRequest_BeginChunk); ok {
return x.BeginChunk
}
return nil
}
func (x *AddArtifactsRequest) GetChunk() *AddArtifactsRequest_ArtifactChunk {
if x, ok := x.GetPayload().(*AddArtifactsRequest_Chunk); ok {
return x.Chunk
}
return nil
}
type isAddArtifactsRequest_Payload interface {
isAddArtifactsRequest_Payload()
}
type AddArtifactsRequest_Batch_ struct {
Batch *AddArtifactsRequest_Batch `protobuf:"bytes,3,opt,name=batch,proto3,oneof"`
}
type AddArtifactsRequest_BeginChunk struct {
// The metadata and the initial chunk of a large artifact chunked into multiple requests.
// The server side is notified about the total size of the large artifact as well as the
// number of chunks to expect.
BeginChunk *AddArtifactsRequest_BeginChunkedArtifact `protobuf:"bytes,4,opt,name=begin_chunk,json=beginChunk,proto3,oneof"`
}
type AddArtifactsRequest_Chunk struct {
// A chunk of an artifact excluding metadata. This can be any chunk of a large artifact
// excluding the first chunk (which is included in `BeginChunkedArtifact`).
Chunk *AddArtifactsRequest_ArtifactChunk `protobuf:"bytes,5,opt,name=chunk,proto3,oneof"`
}
func (*AddArtifactsRequest_Batch_) isAddArtifactsRequest_Payload() {}
func (*AddArtifactsRequest_BeginChunk) isAddArtifactsRequest_Payload() {}
func (*AddArtifactsRequest_Chunk) isAddArtifactsRequest_Payload() {}
// Response to adding an artifact. Contains relevant metadata to verify successful transfer of
// artifact(s).
// Next ID: 4
type AddArtifactsResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Session id in which the AddArtifact was running.
SessionId string `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// The list of artifact(s) seen by the server.
Artifacts []*AddArtifactsResponse_ArtifactSummary `protobuf:"bytes,1,rep,name=artifacts,proto3" json:"artifacts,omitempty"`
}
func (x *AddArtifactsResponse) Reset() {
*x = AddArtifactsResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AddArtifactsResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AddArtifactsResponse) ProtoMessage() {}
func (x *AddArtifactsResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[10]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AddArtifactsResponse.ProtoReflect.Descriptor instead.
func (*AddArtifactsResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{10}
}
func (x *AddArtifactsResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *AddArtifactsResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *AddArtifactsResponse) GetArtifacts() []*AddArtifactsResponse_ArtifactSummary {
if x != nil {
return x.Artifacts
}
return nil
}
// Request to get current statuses of artifacts at the server side.
type ArtifactStatusesRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,5,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,3,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// The name of the artifact is expected in the form of a "Relative Path" that is made up of a
// sequence of directories and the final file element.
// Examples of "Relative Path"s: "jars/test.jar", "classes/xyz.class", "abc.xyz", "a/b/X.jar".
// The server is expected to maintain the hierarchy of files as defined by their name. (i.e
// The relative path of the file on the server's filesystem will be the same as the name of
// the provided artifact)
Names []string `protobuf:"bytes,4,rep,name=names,proto3" json:"names,omitempty"`
}
func (x *ArtifactStatusesRequest) Reset() {
*x = ArtifactStatusesRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[11]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ArtifactStatusesRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ArtifactStatusesRequest) ProtoMessage() {}
func (x *ArtifactStatusesRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[11]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ArtifactStatusesRequest.ProtoReflect.Descriptor instead.
func (*ArtifactStatusesRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{11}
}
func (x *ArtifactStatusesRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ArtifactStatusesRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *ArtifactStatusesRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ArtifactStatusesRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (x *ArtifactStatusesRequest) GetNames() []string {
if x != nil {
return x.Names
}
return nil
}
// Response to checking artifact statuses.
// Next ID: 4
type ArtifactStatusesResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Session id in which the ArtifactStatus was running.
SessionId string `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// A map of artifact names to their statuses.
Statuses 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"`
}
func (x *ArtifactStatusesResponse) Reset() {
*x = ArtifactStatusesResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[12]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ArtifactStatusesResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ArtifactStatusesResponse) ProtoMessage() {}
func (x *ArtifactStatusesResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[12]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ArtifactStatusesResponse.ProtoReflect.Descriptor instead.
func (*ArtifactStatusesResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{12}
}
func (x *ArtifactStatusesResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ArtifactStatusesResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *ArtifactStatusesResponse) GetStatuses() map[string]*ArtifactStatusesResponse_ArtifactStatus {
if x != nil {
return x.Statuses
}
return nil
}
type InterruptRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id specifies a spark session for a user id (which is specified
// by user_context.user_id). The session_id is set by the client to be able to
// collate streaming responses from different queries within the dedicated session.
// The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,3,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// (Required) The type of interrupt to execute.
InterruptType InterruptRequest_InterruptType `protobuf:"varint,4,opt,name=interrupt_type,json=interruptType,proto3,enum=spark.connect.InterruptRequest_InterruptType" json:"interrupt_type,omitempty"`
// Types that are assignable to Interrupt:
//
// *InterruptRequest_OperationTag
// *InterruptRequest_OperationId
Interrupt isInterruptRequest_Interrupt `protobuf_oneof:"interrupt"`
}
func (x *InterruptRequest) Reset() {
*x = InterruptRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[13]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *InterruptRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*InterruptRequest) ProtoMessage() {}
func (x *InterruptRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[13]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use InterruptRequest.ProtoReflect.Descriptor instead.
func (*InterruptRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{13}
}
func (x *InterruptRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *InterruptRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *InterruptRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *InterruptRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (x *InterruptRequest) GetInterruptType() InterruptRequest_InterruptType {
if x != nil {
return x.InterruptType
}
return InterruptRequest_INTERRUPT_TYPE_UNSPECIFIED
}
func (m *InterruptRequest) GetInterrupt() isInterruptRequest_Interrupt {
if m != nil {
return m.Interrupt
}
return nil
}
func (x *InterruptRequest) GetOperationTag() string {
if x, ok := x.GetInterrupt().(*InterruptRequest_OperationTag); ok {
return x.OperationTag
}
return ""
}
func (x *InterruptRequest) GetOperationId() string {
if x, ok := x.GetInterrupt().(*InterruptRequest_OperationId); ok {
return x.OperationId
}
return ""
}
type isInterruptRequest_Interrupt interface {
isInterruptRequest_Interrupt()
}
type InterruptRequest_OperationTag struct {
// if interrupt_tag == INTERRUPT_TYPE_TAG, interrupt operation with this tag.
OperationTag string `protobuf:"bytes,5,opt,name=operation_tag,json=operationTag,proto3,oneof"`
}
type InterruptRequest_OperationId struct {
// if interrupt_tag == INTERRUPT_TYPE_OPERATION_ID, interrupt operation with this operation_id.
OperationId string `protobuf:"bytes,6,opt,name=operation_id,json=operationId,proto3,oneof"`
}
func (*InterruptRequest_OperationTag) isInterruptRequest_Interrupt() {}
func (*InterruptRequest_OperationId) isInterruptRequest_Interrupt() {}
// Next ID: 4
type InterruptResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Session id in which the interrupt was running.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// Operation ids of the executions which were interrupted.
InterruptedIds []string `protobuf:"bytes,2,rep,name=interrupted_ids,json=interruptedIds,proto3" json:"interrupted_ids,omitempty"`
}
func (x *InterruptResponse) Reset() {
*x = InterruptResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[14]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *InterruptResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*InterruptResponse) ProtoMessage() {}
func (x *InterruptResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[14]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use InterruptResponse.ProtoReflect.Descriptor instead.
func (*InterruptResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{14}
}
func (x *InterruptResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *InterruptResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *InterruptResponse) GetInterruptedIds() []string {
if x != nil {
return x.InterruptedIds
}
return nil
}
type ReattachOptions struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// If true, the request can be reattached to using ReattachExecute.
// ReattachExecute can be used either if the stream broke with a GRPC network error,
// or if the server closed the stream without sending a response with StreamStatus.complete=true.
// The server will keep a buffer of responses in case a response is lost, and
// ReattachExecute needs to back-track.
//
// If false, the execution response stream will will not be reattachable, and all responses are
// immediately released by the server after being sent.
Reattachable bool `protobuf:"varint,1,opt,name=reattachable,proto3" json:"reattachable,omitempty"`
}
func (x *ReattachOptions) Reset() {
*x = ReattachOptions{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[15]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReattachOptions) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReattachOptions) ProtoMessage() {}
func (x *ReattachOptions) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[15]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReattachOptions.ProtoReflect.Descriptor instead.
func (*ReattachOptions) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{15}
}
func (x *ReattachOptions) GetReattachable() bool {
if x != nil {
return x.Reattachable
}
return false
}
type ReattachExecuteRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id of the request to reattach to.
// This must be an id of existing session.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,6,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
//
// user_context.user_id and session+id both identify a unique remote spark session on the
// server side.
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Required)
// Provide an id of the request to reattach to.
// This must be an id of existing operation.
OperationId string `protobuf:"bytes,3,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,4,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// (Optional)
// Last already processed response id from the response stream.
// After reattach, server will resume the response stream after that response.
// If not specified, server will restart the stream from the start.
//
// Note: server controls the amount of responses that it buffers and it may drop responses,
// that are far behind the latest returned response, so this can't be used to arbitrarily
// scroll back the cursor. If the response is no longer available, this will result in an error.
LastResponseId *string `protobuf:"bytes,5,opt,name=last_response_id,json=lastResponseId,proto3,oneof" json:"last_response_id,omitempty"`
}
func (x *ReattachExecuteRequest) Reset() {
*x = ReattachExecuteRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[16]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReattachExecuteRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReattachExecuteRequest) ProtoMessage() {}
func (x *ReattachExecuteRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[16]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReattachExecuteRequest.ProtoReflect.Descriptor instead.
func (*ReattachExecuteRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{16}
}
func (x *ReattachExecuteRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ReattachExecuteRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *ReattachExecuteRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ReattachExecuteRequest) GetOperationId() string {
if x != nil {
return x.OperationId
}
return ""
}
func (x *ReattachExecuteRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (x *ReattachExecuteRequest) GetLastResponseId() string {
if x != nil && x.LastResponseId != nil {
return *x.LastResponseId
}
return ""
}
type ReleaseExecuteRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id of the request to reattach to.
// This must be an id of existing session.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,7,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// (Required) User context
//
// user_context.user_id and session+id both identify a unique remote spark session on the
// server side.
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Required)
// Provide an id of the request to reattach to.
// This must be an id of existing operation.
OperationId string `protobuf:"bytes,3,opt,name=operation_id,json=operationId,proto3" json:"operation_id,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,4,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// Types that are assignable to Release:
//
// *ReleaseExecuteRequest_ReleaseAll_
// *ReleaseExecuteRequest_ReleaseUntil_
Release isReleaseExecuteRequest_Release `protobuf_oneof:"release"`
}
func (x *ReleaseExecuteRequest) Reset() {
*x = ReleaseExecuteRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[17]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReleaseExecuteRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReleaseExecuteRequest) ProtoMessage() {}
func (x *ReleaseExecuteRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[17]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReleaseExecuteRequest.ProtoReflect.Descriptor instead.
func (*ReleaseExecuteRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{17}
}
func (x *ReleaseExecuteRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ReleaseExecuteRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *ReleaseExecuteRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ReleaseExecuteRequest) GetOperationId() string {
if x != nil {
return x.OperationId
}
return ""
}
func (x *ReleaseExecuteRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (m *ReleaseExecuteRequest) GetRelease() isReleaseExecuteRequest_Release {
if m != nil {
return m.Release
}
return nil
}
func (x *ReleaseExecuteRequest) GetReleaseAll() *ReleaseExecuteRequest_ReleaseAll {
if x, ok := x.GetRelease().(*ReleaseExecuteRequest_ReleaseAll_); ok {
return x.ReleaseAll
}
return nil
}
func (x *ReleaseExecuteRequest) GetReleaseUntil() *ReleaseExecuteRequest_ReleaseUntil {
if x, ok := x.GetRelease().(*ReleaseExecuteRequest_ReleaseUntil_); ok {
return x.ReleaseUntil
}
return nil
}
type isReleaseExecuteRequest_Release interface {
isReleaseExecuteRequest_Release()
}
type ReleaseExecuteRequest_ReleaseAll_ struct {
ReleaseAll *ReleaseExecuteRequest_ReleaseAll `protobuf:"bytes,5,opt,name=release_all,json=releaseAll,proto3,oneof"`
}
type ReleaseExecuteRequest_ReleaseUntil_ struct {
ReleaseUntil *ReleaseExecuteRequest_ReleaseUntil `protobuf:"bytes,6,opt,name=release_until,json=releaseUntil,proto3,oneof"`
}
func (*ReleaseExecuteRequest_ReleaseAll_) isReleaseExecuteRequest_Release() {}
func (*ReleaseExecuteRequest_ReleaseUntil_) isReleaseExecuteRequest_Release() {}
// Next ID: 4
type ReleaseExecuteResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Session id in which the release was running.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
// Operation id of the operation on which the release executed.
// If the operation couldn't be found (because e.g. it was concurrently released), will be unset.
// Otherwise, it will be equal to the operation_id from request.
OperationId *string `protobuf:"bytes,2,opt,name=operation_id,json=operationId,proto3,oneof" json:"operation_id,omitempty"`
}
func (x *ReleaseExecuteResponse) Reset() {
*x = ReleaseExecuteResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[18]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReleaseExecuteResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReleaseExecuteResponse) ProtoMessage() {}
func (x *ReleaseExecuteResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[18]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReleaseExecuteResponse.ProtoReflect.Descriptor instead.
func (*ReleaseExecuteResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{18}
}
func (x *ReleaseExecuteResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ReleaseExecuteResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *ReleaseExecuteResponse) GetOperationId() string {
if x != nil && x.OperationId != nil {
return *x.OperationId
}
return ""
}
type ReleaseSessionRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
//
// The session_id of the request to reattach to.
// This must be an id of existing session.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Required) User context
//
// user_context.user_id and session+id both identify a unique remote spark session on the
// server side.
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,3,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
// Signals the server to allow the client to reconnect to the session after it is released.
//
// By default, the server tombstones the session upon release, preventing reconnections and
// fully cleaning the session state.
//
// If this flag is set to true, the server may permit the client to reconnect to the session
// post-release, even if the session state has been cleaned. This can result in missing state,
// such as Temporary Views, Temporary UDFs, or the Current Catalog, in the reconnected session.
//
// Use this option sparingly and only when the client fully understands the implications of
// reconnecting to a released session. The client must ensure that any queries executed do not
// rely on the session state prior to its release.
AllowReconnect bool `protobuf:"varint,4,opt,name=allow_reconnect,json=allowReconnect,proto3" json:"allow_reconnect,omitempty"`
}
func (x *ReleaseSessionRequest) Reset() {
*x = ReleaseSessionRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[19]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReleaseSessionRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReleaseSessionRequest) ProtoMessage() {}
func (x *ReleaseSessionRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[19]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReleaseSessionRequest.ProtoReflect.Descriptor instead.
func (*ReleaseSessionRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{19}
}
func (x *ReleaseSessionRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ReleaseSessionRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *ReleaseSessionRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
func (x *ReleaseSessionRequest) GetAllowReconnect() bool {
if x != nil {
return x.AllowReconnect
}
return false
}
// Next ID: 3
type ReleaseSessionResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Session id of the session on which the release executed.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,2,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
}
func (x *ReleaseSessionResponse) Reset() {
*x = ReleaseSessionResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[20]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReleaseSessionResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReleaseSessionResponse) ProtoMessage() {}
func (x *ReleaseSessionResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[20]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ReleaseSessionResponse.ProtoReflect.Descriptor instead.
func (*ReleaseSessionResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{20}
}
func (x *ReleaseSessionResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *ReleaseSessionResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
type FetchErrorDetailsRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required)
// The session_id specifies a Spark session for a user identified by user_context.user_id.
// The id should be a UUID string of the format `00112233-4455-6677-8899-aabbccddeeff`.
SessionId string `protobuf:"bytes,1,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// (Optional)
//
// Server-side generated idempotency key from the previous responses (if any). Server
// can use this to validate that the server side session has not changed.
ClientObservedServerSideSessionId *string `protobuf:"bytes,5,opt,name=client_observed_server_side_session_id,json=clientObservedServerSideSessionId,proto3,oneof" json:"client_observed_server_side_session_id,omitempty"`
// User context
UserContext *UserContext `protobuf:"bytes,2,opt,name=user_context,json=userContext,proto3" json:"user_context,omitempty"`
// (Required)
// The id of the error.
ErrorId string `protobuf:"bytes,3,opt,name=error_id,json=errorId,proto3" json:"error_id,omitempty"`
// Provides optional information about the client sending the request. This field
// can be used for language or version specific information and is only intended for
// logging purposes and will not be interpreted by the server.
ClientType *string `protobuf:"bytes,4,opt,name=client_type,json=clientType,proto3,oneof" json:"client_type,omitempty"`
}
func (x *FetchErrorDetailsRequest) Reset() {
*x = FetchErrorDetailsRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[21]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *FetchErrorDetailsRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*FetchErrorDetailsRequest) ProtoMessage() {}
func (x *FetchErrorDetailsRequest) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[21]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use FetchErrorDetailsRequest.ProtoReflect.Descriptor instead.
func (*FetchErrorDetailsRequest) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{21}
}
func (x *FetchErrorDetailsRequest) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *FetchErrorDetailsRequest) GetClientObservedServerSideSessionId() string {
if x != nil && x.ClientObservedServerSideSessionId != nil {
return *x.ClientObservedServerSideSessionId
}
return ""
}
func (x *FetchErrorDetailsRequest) GetUserContext() *UserContext {
if x != nil {
return x.UserContext
}
return nil
}
func (x *FetchErrorDetailsRequest) GetErrorId() string {
if x != nil {
return x.ErrorId
}
return ""
}
func (x *FetchErrorDetailsRequest) GetClientType() string {
if x != nil && x.ClientType != nil {
return *x.ClientType
}
return ""
}
// Next ID: 5
type FetchErrorDetailsResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Server-side generated idempotency key that the client can use to assert that the server side
// session has not changed.
ServerSideSessionId string `protobuf:"bytes,3,opt,name=server_side_session_id,json=serverSideSessionId,proto3" json:"server_side_session_id,omitempty"`
SessionId string `protobuf:"bytes,4,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// The index of the root error in errors. The field will not be set if the error is not found.
RootErrorIdx *int32 `protobuf:"varint,1,opt,name=root_error_idx,json=rootErrorIdx,proto3,oneof" json:"root_error_idx,omitempty"`
// A list of errors.
Errors []*FetchErrorDetailsResponse_Error `protobuf:"bytes,2,rep,name=errors,proto3" json:"errors,omitempty"`
}
func (x *FetchErrorDetailsResponse) Reset() {
*x = FetchErrorDetailsResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[22]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *FetchErrorDetailsResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*FetchErrorDetailsResponse) ProtoMessage() {}
func (x *FetchErrorDetailsResponse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[22]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use FetchErrorDetailsResponse.ProtoReflect.Descriptor instead.
func (*FetchErrorDetailsResponse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{22}
}
func (x *FetchErrorDetailsResponse) GetServerSideSessionId() string {
if x != nil {
return x.ServerSideSessionId
}
return ""
}
func (x *FetchErrorDetailsResponse) GetSessionId() string {
if x != nil {
return x.SessionId
}
return ""
}
func (x *FetchErrorDetailsResponse) GetRootErrorIdx() int32 {
if x != nil && x.RootErrorIdx != nil {
return *x.RootErrorIdx
}
return 0
}
func (x *FetchErrorDetailsResponse) GetErrors() []*FetchErrorDetailsResponse_Error {
if x != nil {
return x.Errors
}
return nil
}
type CheckpointCommandResult struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan checkpointed.
Relation *CachedRemoteRelation `protobuf:"bytes,1,opt,name=relation,proto3" json:"relation,omitempty"`
}
func (x *CheckpointCommandResult) Reset() {
*x = CheckpointCommandResult{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[23]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *CheckpointCommandResult) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*CheckpointCommandResult) ProtoMessage() {}
func (x *CheckpointCommandResult) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[23]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use CheckpointCommandResult.ProtoReflect.Descriptor instead.
func (*CheckpointCommandResult) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{23}
}
func (x *CheckpointCommandResult) GetRelation() *CachedRemoteRelation {
if x != nil {
return x.Relation
}
return nil
}
type AnalyzePlanRequest_Schema struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
}
func (x *AnalyzePlanRequest_Schema) Reset() {
*x = AnalyzePlanRequest_Schema{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[24]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_Schema) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_Schema) ProtoMessage() {}
func (x *AnalyzePlanRequest_Schema) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[24]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_Schema.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_Schema) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 0}
}
func (x *AnalyzePlanRequest_Schema) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
// Explains the input plan based on a configurable mode.
type AnalyzePlanRequest_Explain struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
// (Required) For analyzePlan rpc calls, configure the mode to explain plan in strings.
ExplainMode AnalyzePlanRequest_Explain_ExplainMode `protobuf:"varint,2,opt,name=explain_mode,json=explainMode,proto3,enum=spark.connect.AnalyzePlanRequest_Explain_ExplainMode" json:"explain_mode,omitempty"`
}
func (x *AnalyzePlanRequest_Explain) Reset() {
*x = AnalyzePlanRequest_Explain{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[25]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_Explain) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_Explain) ProtoMessage() {}
func (x *AnalyzePlanRequest_Explain) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[25]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_Explain.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_Explain) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 1}
}
func (x *AnalyzePlanRequest_Explain) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
func (x *AnalyzePlanRequest_Explain) GetExplainMode() AnalyzePlanRequest_Explain_ExplainMode {
if x != nil {
return x.ExplainMode
}
return AnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED
}
type AnalyzePlanRequest_TreeString struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
// (Optional) Max level of the schema.
Level *int32 `protobuf:"varint,2,opt,name=level,proto3,oneof" json:"level,omitempty"`
}
func (x *AnalyzePlanRequest_TreeString) Reset() {
*x = AnalyzePlanRequest_TreeString{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[26]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_TreeString) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_TreeString) ProtoMessage() {}
func (x *AnalyzePlanRequest_TreeString) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[26]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_TreeString.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_TreeString) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 2}
}
func (x *AnalyzePlanRequest_TreeString) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
func (x *AnalyzePlanRequest_TreeString) GetLevel() int32 {
if x != nil && x.Level != nil {
return *x.Level
}
return 0
}
type AnalyzePlanRequest_IsLocal struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
}
func (x *AnalyzePlanRequest_IsLocal) Reset() {
*x = AnalyzePlanRequest_IsLocal{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[27]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_IsLocal) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_IsLocal) ProtoMessage() {}
func (x *AnalyzePlanRequest_IsLocal) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[27]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_IsLocal.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_IsLocal) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 3}
}
func (x *AnalyzePlanRequest_IsLocal) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
type AnalyzePlanRequest_IsStreaming struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
}
func (x *AnalyzePlanRequest_IsStreaming) Reset() {
*x = AnalyzePlanRequest_IsStreaming{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[28]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_IsStreaming) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_IsStreaming) ProtoMessage() {}
func (x *AnalyzePlanRequest_IsStreaming) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[28]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_IsStreaming.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_IsStreaming) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 4}
}
func (x *AnalyzePlanRequest_IsStreaming) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
type AnalyzePlanRequest_InputFiles struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to be analyzed.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
}
func (x *AnalyzePlanRequest_InputFiles) Reset() {
*x = AnalyzePlanRequest_InputFiles{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_InputFiles) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_InputFiles) ProtoMessage() {}
func (x *AnalyzePlanRequest_InputFiles) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[29]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_InputFiles.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_InputFiles) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 5}
}
func (x *AnalyzePlanRequest_InputFiles) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
type AnalyzePlanRequest_SparkVersion struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
}
func (x *AnalyzePlanRequest_SparkVersion) Reset() {
*x = AnalyzePlanRequest_SparkVersion{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_SparkVersion) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_SparkVersion) ProtoMessage() {}
func (x *AnalyzePlanRequest_SparkVersion) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[30]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_SparkVersion.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_SparkVersion) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 6}
}
type AnalyzePlanRequest_DDLParse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The DDL formatted string to be parsed.
DdlString string `protobuf:"bytes,1,opt,name=ddl_string,json=ddlString,proto3" json:"ddl_string,omitempty"`
}
func (x *AnalyzePlanRequest_DDLParse) Reset() {
*x = AnalyzePlanRequest_DDLParse{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_DDLParse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_DDLParse) ProtoMessage() {}
func (x *AnalyzePlanRequest_DDLParse) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[31]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_DDLParse.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_DDLParse) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 7}
}
func (x *AnalyzePlanRequest_DDLParse) GetDdlString() string {
if x != nil {
return x.DdlString
}
return ""
}
// Returns `true` when the logical query plans are equal and therefore return same results.
type AnalyzePlanRequest_SameSemantics struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The plan to be compared.
TargetPlan *Plan `protobuf:"bytes,1,opt,name=target_plan,json=targetPlan,proto3" json:"target_plan,omitempty"`
// (Required) The other plan to be compared.
OtherPlan *Plan `protobuf:"bytes,2,opt,name=other_plan,json=otherPlan,proto3" json:"other_plan,omitempty"`
}
func (x *AnalyzePlanRequest_SameSemantics) Reset() {
*x = AnalyzePlanRequest_SameSemantics{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_SameSemantics) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_SameSemantics) ProtoMessage() {}
func (x *AnalyzePlanRequest_SameSemantics) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_SameSemantics.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_SameSemantics) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 8}
}
func (x *AnalyzePlanRequest_SameSemantics) GetTargetPlan() *Plan {
if x != nil {
return x.TargetPlan
}
return nil
}
func (x *AnalyzePlanRequest_SameSemantics) GetOtherPlan() *Plan {
if x != nil {
return x.OtherPlan
}
return nil
}
type AnalyzePlanRequest_SemanticHash struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to get a hashCode.
Plan *Plan `protobuf:"bytes,1,opt,name=plan,proto3" json:"plan,omitempty"`
}
func (x *AnalyzePlanRequest_SemanticHash) Reset() {
*x = AnalyzePlanRequest_SemanticHash{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_SemanticHash) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_SemanticHash) ProtoMessage() {}
func (x *AnalyzePlanRequest_SemanticHash) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_SemanticHash.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_SemanticHash) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 9}
}
func (x *AnalyzePlanRequest_SemanticHash) GetPlan() *Plan {
if x != nil {
return x.Plan
}
return nil
}
type AnalyzePlanRequest_Persist struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to persist.
Relation *Relation `protobuf:"bytes,1,opt,name=relation,proto3" json:"relation,omitempty"`
// (Optional) The storage level.
StorageLevel *StorageLevel `protobuf:"bytes,2,opt,name=storage_level,json=storageLevel,proto3,oneof" json:"storage_level,omitempty"`
}
func (x *AnalyzePlanRequest_Persist) Reset() {
*x = AnalyzePlanRequest_Persist{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_Persist) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_Persist) ProtoMessage() {}
func (x *AnalyzePlanRequest_Persist) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_Persist.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_Persist) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 10}
}
func (x *AnalyzePlanRequest_Persist) GetRelation() *Relation {
if x != nil {
return x.Relation
}
return nil
}
func (x *AnalyzePlanRequest_Persist) GetStorageLevel() *StorageLevel {
if x != nil {
return x.StorageLevel
}
return nil
}
type AnalyzePlanRequest_Unpersist struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to unpersist.
Relation *Relation `protobuf:"bytes,1,opt,name=relation,proto3" json:"relation,omitempty"`
// (Optional) Whether to block until all blocks are deleted.
Blocking *bool `protobuf:"varint,2,opt,name=blocking,proto3,oneof" json:"blocking,omitempty"`
}
func (x *AnalyzePlanRequest_Unpersist) Reset() {
*x = AnalyzePlanRequest_Unpersist{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[35]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_Unpersist) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_Unpersist) ProtoMessage() {}
func (x *AnalyzePlanRequest_Unpersist) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[35]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use AnalyzePlanRequest_Unpersist.ProtoReflect.Descriptor instead.
func (*AnalyzePlanRequest_Unpersist) Descriptor() ([]byte, []int) {
return file_spark_connect_base_proto_rawDescGZIP(), []int{2, 11}
}
func (x *AnalyzePlanRequest_Unpersist) GetRelation() *Relation {
if x != nil {
return x.Relation
}
return nil
}
func (x *AnalyzePlanRequest_Unpersist) GetBlocking() bool {
if x != nil && x.Blocking != nil {
return *x.Blocking
}
return false
}
type AnalyzePlanRequest_GetStorageLevel struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// (Required) The logical plan to get the storage level.
Relation *Relation `protobuf:"bytes,1,opt,name=relation,proto3" json:"relation,omitempty"`
}
func (x *AnalyzePlanRequest_GetStorageLevel) Reset() {
*x = AnalyzePlanRequest_GetStorageLevel{}
if protoimpl.UnsafeEnabled {
mi := &file_spark_connect_base_proto_msgTypes[36]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *AnalyzePlanRequest_GetStorageLevel) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*AnalyzePlanRequest_GetStorageLevel) ProtoMessage() {}
func (x *AnalyzePlanRequest_GetStorageLevel) ProtoReflect() protoreflect.Message {
mi := &file_spark_connect_base_proto_msgTypes[36]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
gitextract_ljoq267_/
├── .asf.yaml
├── .github/
│ ├── PULL_REQUEST_TEMPLATE
│ ├── dependabot.yml
│ └── workflows/
│ └── build.yml
├── .gitignore
├── .gitmodules
├── .golangci.yml
├── CONTRIBUTING.md
├── LICENSE
├── Makefile
├── NOTICE
├── README.md
├── buf.gen.yaml
├── buf.work.yaml
├── cmd/
│ ├── spark-connect-example-raw-grpc-client/
│ │ └── main.go
│ └── spark-connect-example-spark-session/
│ └── main.go
├── dev/
│ ├── .rat-excludes
│ ├── README.md
│ ├── check-license
│ ├── gen.py
│ ├── release.py
│ └── requirements.txt
├── go.mod
├── go.sum
├── internal/
│ ├── generated/
│ │ ├── base.pb.go
│ │ ├── base_grpc.pb.go
│ │ ├── catalog.pb.go
│ │ ├── commands.pb.go
│ │ ├── common.pb.go
│ │ ├── example_plugins.pb.go
│ │ ├── expressions.pb.go
│ │ ├── ml.pb.go
│ │ ├── ml_common.pb.go
│ │ ├── pipelines.pb.go
│ │ ├── relations.pb.go
│ │ └── types.pb.go
│ └── tests/
│ └── integration/
│ ├── dataframe_test.go
│ ├── functions_test.go
│ ├── helper.go
│ ├── spark_runner.go
│ └── sql_test.go
├── java/
│ ├── .gitignore
│ ├── README.md
│ ├── build.sbt
│ ├── run.sh
│ └── src/
│ └── main/
│ └── scala/
│ └── org/
│ └── apache/
│ └── spark/
│ └── golang/
│ └── Runner.scala
├── merge_connect_go_pr.py
├── quick-start.md
└── spark/
├── client/
│ ├── base/
│ │ └── base.go
│ ├── channel/
│ │ ├── channel.go
│ │ ├── channel_test.go
│ │ └── compat.go
│ ├── client.go
│ ├── client_test.go
│ ├── conf.go
│ ├── options/
│ │ └── options.go
│ ├── retry.go
│ ├── retry_test.go
│ └── testutils/
│ └── utils.go
├── mocks/
│ ├── mock_executor.go
│ └── mocks.go
├── sparkerrors/
│ ├── errors.go
│ └── errors_test.go
├── sql/
│ ├── column/
│ │ ├── column.go
│ │ ├── column_test.go
│ │ ├── expressions.go
│ │ └── expressions_test.go
│ ├── dataframe.go
│ ├── dataframe_test.go
│ ├── dataframenafunctions.go
│ ├── dataframereader.go
│ ├── dataframereader_test.go
│ ├── dataframestatfunctions.go
│ ├── dataframewriter.go
│ ├── dataframewriter_test.go
│ ├── executeplanclient.go
│ ├── functions/
│ │ ├── buiitins.go
│ │ └── generated.go
│ ├── group.go
│ ├── group_test.go
│ ├── mocks_test.go
│ ├── plan.go
│ ├── plan_test.go
│ ├── sparksession.go
│ ├── sparksession_integration_test.go
│ ├── sparksession_test.go
│ ├── types/
│ │ ├── arrow.go
│ │ ├── arrow_test.go
│ │ ├── builtin.go
│ │ ├── builtin_test.go
│ │ ├── conversion.go
│ │ ├── conversion_test.go
│ │ ├── datatype.go
│ │ ├── datatype_test.go
│ │ ├── row.go
│ │ ├── row_json_test.go
│ │ ├── row_test.go
│ │ ├── structtype.go
│ │ └── structtype_test.go
│ └── utils/
│ ├── check.go
│ ├── check_test.go
│ ├── consts.go
│ └── consts_test.go
└── version.go
Showing preview only (489K chars total). Download the full file or copy to clipboard to get everything.
SYMBOL INDEX (5233 symbols across 75 files)
FILE: cmd/spark-connect-example-raw-grpc-client/main.go
function main (line 33) | func main() {
FILE: cmd/spark-connect-example-spark-session/main.go
function main (line 41) | func main() {
FILE: dev/gen.py
function normalize (line 28) | def normalize(input: str) -> str:
FILE: dev/release.py
function run_command (line 32) | def run_command(cmd: List[str], cwd: str = None, check: bool = True) -> ...
function get_commits_between_tags (line 46) | def get_commits_between_tags(repo_path: str, previous_tag: str, commit_s...
function create_release_notes (line 69) | def create_release_notes(commits: List[Dict[str, str]]) -> str:
function verify_gpg_key (line 81) | def verify_gpg_key(gpg_user: str) -> bool:
function sign_file (line 90) | def sign_file(file_path: str, gpg_user: str) -> str:
function verify_signature (line 110) | def verify_signature(file_path: str, signature_path: str) -> bool:
function download_file (line 119) | def download_file(url: str, local_path: str):
function upload_release_asset (line 131) | def upload_release_asset(release, file_path: str):
function main (line 142) | def main():
FILE: internal/generated/base.pb.go
constant _ (line 35) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 37) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type AnalyzePlanRequest_Explain_ExplainMode (line 41) | type AnalyzePlanRequest_Explain_ExplainMode
method Enum (line 81) | func (x AnalyzePlanRequest_Explain_ExplainMode) Enum() *AnalyzePlanReq...
method String (line 87) | func (x AnalyzePlanRequest_Explain_ExplainMode) String() string {
method Descriptor (line 91) | func (AnalyzePlanRequest_Explain_ExplainMode) Descriptor() protoreflec...
method Type (line 95) | func (AnalyzePlanRequest_Explain_ExplainMode) Type() protoreflect.Enum...
method Number (line 99) | func (x AnalyzePlanRequest_Explain_ExplainMode) Number() protoreflect....
method EnumDescriptor (line 104) | func (AnalyzePlanRequest_Explain_ExplainMode) EnumDescriptor() ([]byte...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED (line 44) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_UNSPECIFIED AnalyzePlanRequest_E...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_SIMPLE (line 46) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_SIMPLE AnalyzePlanRequest_Explai...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_EXTENDED (line 52) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_EXTENDED AnalyzePlanRequest_Expl...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_CODEGEN (line 54) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_CODEGEN AnalyzePlanRequest_Expla...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_COST (line 56) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_COST AnalyzePlanRequest_Explain_...
constant AnalyzePlanRequest_Explain_EXPLAIN_MODE_FORMATTED (line 58) | AnalyzePlanRequest_Explain_EXPLAIN_MODE_FORMATTED AnalyzePlanRequest_Exp...
type InterruptRequest_InterruptType (line 108) | type InterruptRequest_InterruptType
method Enum (line 136) | func (x InterruptRequest_InterruptType) Enum() *InterruptRequest_Inter...
method String (line 142) | func (x InterruptRequest_InterruptType) String() string {
method Descriptor (line 146) | func (InterruptRequest_InterruptType) Descriptor() protoreflect.EnumDe...
method Type (line 150) | func (InterruptRequest_InterruptType) Type() protoreflect.EnumType {
method Number (line 154) | func (x InterruptRequest_InterruptType) Number() protoreflect.EnumNumb...
method EnumDescriptor (line 159) | func (InterruptRequest_InterruptType) EnumDescriptor() ([]byte, []int) {
constant InterruptRequest_INTERRUPT_TYPE_UNSPECIFIED (line 111) | InterruptRequest_INTERRUPT_TYPE_UNSPECIFIED InterruptRequest_InterruptTy...
constant InterruptRequest_INTERRUPT_TYPE_ALL (line 113) | InterruptRequest_INTERRUPT_TYPE_ALL InterruptRequest_InterruptType = 1
constant InterruptRequest_INTERRUPT_TYPE_TAG (line 115) | InterruptRequest_INTERRUPT_TYPE_TAG InterruptRequest_InterruptType = 2
constant InterruptRequest_INTERRUPT_TYPE_OPERATION_ID (line 117) | InterruptRequest_INTERRUPT_TYPE_OPERATION_ID InterruptRequest_InterruptT...
type FetchErrorDetailsResponse_QueryContext_ContextType (line 164) | type FetchErrorDetailsResponse_QueryContext_ContextType
method Enum (line 183) | func (x FetchErrorDetailsResponse_QueryContext_ContextType) Enum() *Fe...
method String (line 189) | func (x FetchErrorDetailsResponse_QueryContext_ContextType) String() s...
method Descriptor (line 193) | func (FetchErrorDetailsResponse_QueryContext_ContextType) Descriptor()...
method Type (line 197) | func (FetchErrorDetailsResponse_QueryContext_ContextType) Type() proto...
method Number (line 201) | func (x FetchErrorDetailsResponse_QueryContext_ContextType) Number() p...
method EnumDescriptor (line 206) | func (FetchErrorDetailsResponse_QueryContext_ContextType) EnumDescript...
constant FetchErrorDetailsResponse_QueryContext_SQL (line 167) | FetchErrorDetailsResponse_QueryContext_SQL FetchErrorDetailsRespon...
constant FetchErrorDetailsResponse_QueryContext_DATAFRAME (line 168) | FetchErrorDetailsResponse_QueryContext_DATAFRAME FetchErrorDetailsRespon...
type Plan (line 214) | type Plan struct
method Reset (line 226) | func (x *Plan) Reset() {
method String (line 235) | func (x *Plan) String() string {
method ProtoMessage (line 239) | func (*Plan) ProtoMessage() {}
method ProtoReflect (line 241) | func (x *Plan) ProtoReflect() protoreflect.Message {
method Descriptor (line 254) | func (*Plan) Descriptor() ([]byte, []int) {
method GetOpType (line 258) | func (m *Plan) GetOpType() isPlan_OpType {
method GetRoot (line 265) | func (x *Plan) GetRoot() *Relation {
method GetCommand (line 272) | func (x *Plan) GetCommand() *Command {
type isPlan_OpType (line 279) | type isPlan_OpType interface
type Plan_Root (line 283) | type Plan_Root struct
method isPlan_OpType (line 291) | func (*Plan_Root) isPlan_OpType() {}
type Plan_Command (line 287) | type Plan_Command struct
method isPlan_OpType (line 293) | func (*Plan_Command) isPlan_OpType() {}
type UserContext (line 297) | type UserContext struct
method Reset (line 311) | func (x *UserContext) Reset() {
method String (line 320) | func (x *UserContext) String() string {
method ProtoMessage (line 324) | func (*UserContext) ProtoMessage() {}
method ProtoReflect (line 326) | func (x *UserContext) ProtoReflect() protoreflect.Message {
method Descriptor (line 339) | func (*UserContext) Descriptor() ([]byte, []int) {
method GetUserId (line 343) | func (x *UserContext) GetUserId() string {
method GetUserName (line 350) | func (x *UserContext) GetUserName() string {
method GetExtensions (line 357) | func (x *UserContext) GetExtensions() []*anypb.Any {
type AnalyzePlanRequest (line 365) | type AnalyzePlanRequest struct
method Reset (line 407) | func (x *AnalyzePlanRequest) Reset() {
method String (line 416) | func (x *AnalyzePlanRequest) String() string {
method ProtoMessage (line 420) | func (*AnalyzePlanRequest) ProtoMessage() {}
method ProtoReflect (line 422) | func (x *AnalyzePlanRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 435) | func (*AnalyzePlanRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 439) | func (x *AnalyzePlanRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 446) | func (x *AnalyzePlanRequest) GetClientObservedServerSideSessionId() st...
method GetUserContext (line 453) | func (x *AnalyzePlanRequest) GetUserContext() *UserContext {
method GetClientType (line 460) | func (x *AnalyzePlanRequest) GetClientType() string {
method GetAnalyze (line 467) | func (m *AnalyzePlanRequest) GetAnalyze() isAnalyzePlanRequest_Analyze {
method GetSchema (line 474) | func (x *AnalyzePlanRequest) GetSchema() *AnalyzePlanRequest_Schema {
method GetExplain (line 481) | func (x *AnalyzePlanRequest) GetExplain() *AnalyzePlanRequest_Explain {
method GetTreeString (line 488) | func (x *AnalyzePlanRequest) GetTreeString() *AnalyzePlanRequest_TreeS...
method GetIsLocal (line 495) | func (x *AnalyzePlanRequest) GetIsLocal() *AnalyzePlanRequest_IsLocal {
method GetIsStreaming (line 502) | func (x *AnalyzePlanRequest) GetIsStreaming() *AnalyzePlanRequest_IsSt...
method GetInputFiles (line 509) | func (x *AnalyzePlanRequest) GetInputFiles() *AnalyzePlanRequest_Input...
method GetSparkVersion (line 516) | func (x *AnalyzePlanRequest) GetSparkVersion() *AnalyzePlanRequest_Spa...
method GetDdlParse (line 523) | func (x *AnalyzePlanRequest) GetDdlParse() *AnalyzePlanRequest_DDLParse {
method GetSameSemantics (line 530) | func (x *AnalyzePlanRequest) GetSameSemantics() *AnalyzePlanRequest_Sa...
method GetSemanticHash (line 537) | func (x *AnalyzePlanRequest) GetSemanticHash() *AnalyzePlanRequest_Sem...
method GetPersist (line 544) | func (x *AnalyzePlanRequest) GetPersist() *AnalyzePlanRequest_Persist {
method GetUnpersist (line 551) | func (x *AnalyzePlanRequest) GetUnpersist() *AnalyzePlanRequest_Unpers...
method GetGetStorageLevel (line 558) | func (x *AnalyzePlanRequest) GetGetStorageLevel() *AnalyzePlanRequest_...
method GetJsonToDdl (line 565) | func (x *AnalyzePlanRequest) GetJsonToDdl() *AnalyzePlanRequest_JsonTo...
type isAnalyzePlanRequest_Analyze (line 572) | type isAnalyzePlanRequest_Analyze interface
type AnalyzePlanRequest_Schema_ (line 576) | type AnalyzePlanRequest_Schema_ struct
method isAnalyzePlanRequest_Analyze (line 632) | func (*AnalyzePlanRequest_Schema_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_Explain_ (line 580) | type AnalyzePlanRequest_Explain_ struct
method isAnalyzePlanRequest_Analyze (line 634) | func (*AnalyzePlanRequest_Explain_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_TreeString_ (line 584) | type AnalyzePlanRequest_TreeString_ struct
method isAnalyzePlanRequest_Analyze (line 636) | func (*AnalyzePlanRequest_TreeString_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_IsLocal_ (line 588) | type AnalyzePlanRequest_IsLocal_ struct
method isAnalyzePlanRequest_Analyze (line 638) | func (*AnalyzePlanRequest_IsLocal_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_IsStreaming_ (line 592) | type AnalyzePlanRequest_IsStreaming_ struct
method isAnalyzePlanRequest_Analyze (line 640) | func (*AnalyzePlanRequest_IsStreaming_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_InputFiles_ (line 596) | type AnalyzePlanRequest_InputFiles_ struct
method isAnalyzePlanRequest_Analyze (line 642) | func (*AnalyzePlanRequest_InputFiles_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_SparkVersion_ (line 600) | type AnalyzePlanRequest_SparkVersion_ struct
method isAnalyzePlanRequest_Analyze (line 644) | func (*AnalyzePlanRequest_SparkVersion_) isAnalyzePlanRequest_Analyze(...
type AnalyzePlanRequest_DdlParse (line 604) | type AnalyzePlanRequest_DdlParse struct
method isAnalyzePlanRequest_Analyze (line 646) | func (*AnalyzePlanRequest_DdlParse) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_SameSemantics_ (line 608) | type AnalyzePlanRequest_SameSemantics_ struct
method isAnalyzePlanRequest_Analyze (line 648) | func (*AnalyzePlanRequest_SameSemantics_) isAnalyzePlanRequest_Analyze...
type AnalyzePlanRequest_SemanticHash_ (line 612) | type AnalyzePlanRequest_SemanticHash_ struct
method isAnalyzePlanRequest_Analyze (line 650) | func (*AnalyzePlanRequest_SemanticHash_) isAnalyzePlanRequest_Analyze(...
type AnalyzePlanRequest_Persist_ (line 616) | type AnalyzePlanRequest_Persist_ struct
method isAnalyzePlanRequest_Analyze (line 652) | func (*AnalyzePlanRequest_Persist_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_Unpersist_ (line 620) | type AnalyzePlanRequest_Unpersist_ struct
method isAnalyzePlanRequest_Analyze (line 654) | func (*AnalyzePlanRequest_Unpersist_) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanRequest_GetStorageLevel_ (line 624) | type AnalyzePlanRequest_GetStorageLevel_ struct
method isAnalyzePlanRequest_Analyze (line 656) | func (*AnalyzePlanRequest_GetStorageLevel_) isAnalyzePlanRequest_Analy...
type AnalyzePlanRequest_JsonToDdl (line 628) | type AnalyzePlanRequest_JsonToDdl struct
method isAnalyzePlanRequest_Analyze (line 658) | func (*AnalyzePlanRequest_JsonToDdl) isAnalyzePlanRequest_Analyze() {}
type AnalyzePlanResponse (line 663) | type AnalyzePlanResponse struct
method Reset (line 691) | func (x *AnalyzePlanResponse) Reset() {
method String (line 700) | func (x *AnalyzePlanResponse) String() string {
method ProtoMessage (line 704) | func (*AnalyzePlanResponse) ProtoMessage() {}
method ProtoReflect (line 706) | func (x *AnalyzePlanResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 719) | func (*AnalyzePlanResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 723) | func (x *AnalyzePlanResponse) GetSessionId() string {
method GetServerSideSessionId (line 730) | func (x *AnalyzePlanResponse) GetServerSideSessionId() string {
method GetResult (line 737) | func (m *AnalyzePlanResponse) GetResult() isAnalyzePlanResponse_Result {
method GetSchema (line 744) | func (x *AnalyzePlanResponse) GetSchema() *AnalyzePlanResponse_Schema {
method GetExplain (line 751) | func (x *AnalyzePlanResponse) GetExplain() *AnalyzePlanResponse_Explain {
method GetTreeString (line 758) | func (x *AnalyzePlanResponse) GetTreeString() *AnalyzePlanResponse_Tre...
method GetIsLocal (line 765) | func (x *AnalyzePlanResponse) GetIsLocal() *AnalyzePlanResponse_IsLocal {
method GetIsStreaming (line 772) | func (x *AnalyzePlanResponse) GetIsStreaming() *AnalyzePlanResponse_Is...
method GetInputFiles (line 779) | func (x *AnalyzePlanResponse) GetInputFiles() *AnalyzePlanResponse_Inp...
method GetSparkVersion (line 786) | func (x *AnalyzePlanResponse) GetSparkVersion() *AnalyzePlanResponse_S...
method GetDdlParse (line 793) | func (x *AnalyzePlanResponse) GetDdlParse() *AnalyzePlanResponse_DDLPa...
method GetSameSemantics (line 800) | func (x *AnalyzePlanResponse) GetSameSemantics() *AnalyzePlanResponse_...
method GetSemanticHash (line 807) | func (x *AnalyzePlanResponse) GetSemanticHash() *AnalyzePlanResponse_S...
method GetPersist (line 814) | func (x *AnalyzePlanResponse) GetPersist() *AnalyzePlanResponse_Persist {
method GetUnpersist (line 821) | func (x *AnalyzePlanResponse) GetUnpersist() *AnalyzePlanResponse_Unpe...
method GetGetStorageLevel (line 828) | func (x *AnalyzePlanResponse) GetGetStorageLevel() *AnalyzePlanRespons...
method GetJsonToDdl (line 835) | func (x *AnalyzePlanResponse) GetJsonToDdl() *AnalyzePlanResponse_Json...
type isAnalyzePlanResponse_Result (line 842) | type isAnalyzePlanResponse_Result interface
type AnalyzePlanResponse_Schema_ (line 846) | type AnalyzePlanResponse_Schema_ struct
method isAnalyzePlanResponse_Result (line 902) | func (*AnalyzePlanResponse_Schema_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_Explain_ (line 850) | type AnalyzePlanResponse_Explain_ struct
method isAnalyzePlanResponse_Result (line 904) | func (*AnalyzePlanResponse_Explain_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_TreeString_ (line 854) | type AnalyzePlanResponse_TreeString_ struct
method isAnalyzePlanResponse_Result (line 906) | func (*AnalyzePlanResponse_TreeString_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_IsLocal_ (line 858) | type AnalyzePlanResponse_IsLocal_ struct
method isAnalyzePlanResponse_Result (line 908) | func (*AnalyzePlanResponse_IsLocal_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_IsStreaming_ (line 862) | type AnalyzePlanResponse_IsStreaming_ struct
method isAnalyzePlanResponse_Result (line 910) | func (*AnalyzePlanResponse_IsStreaming_) isAnalyzePlanResponse_Result(...
type AnalyzePlanResponse_InputFiles_ (line 866) | type AnalyzePlanResponse_InputFiles_ struct
method isAnalyzePlanResponse_Result (line 912) | func (*AnalyzePlanResponse_InputFiles_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_SparkVersion_ (line 870) | type AnalyzePlanResponse_SparkVersion_ struct
method isAnalyzePlanResponse_Result (line 914) | func (*AnalyzePlanResponse_SparkVersion_) isAnalyzePlanResponse_Result...
type AnalyzePlanResponse_DdlParse (line 874) | type AnalyzePlanResponse_DdlParse struct
method isAnalyzePlanResponse_Result (line 916) | func (*AnalyzePlanResponse_DdlParse) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_SameSemantics_ (line 878) | type AnalyzePlanResponse_SameSemantics_ struct
method isAnalyzePlanResponse_Result (line 918) | func (*AnalyzePlanResponse_SameSemantics_) isAnalyzePlanResponse_Resul...
type AnalyzePlanResponse_SemanticHash_ (line 882) | type AnalyzePlanResponse_SemanticHash_ struct
method isAnalyzePlanResponse_Result (line 920) | func (*AnalyzePlanResponse_SemanticHash_) isAnalyzePlanResponse_Result...
type AnalyzePlanResponse_Persist_ (line 886) | type AnalyzePlanResponse_Persist_ struct
method isAnalyzePlanResponse_Result (line 922) | func (*AnalyzePlanResponse_Persist_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_Unpersist_ (line 890) | type AnalyzePlanResponse_Unpersist_ struct
method isAnalyzePlanResponse_Result (line 924) | func (*AnalyzePlanResponse_Unpersist_) isAnalyzePlanResponse_Result() {}
type AnalyzePlanResponse_GetStorageLevel_ (line 894) | type AnalyzePlanResponse_GetStorageLevel_ struct
method isAnalyzePlanResponse_Result (line 926) | func (*AnalyzePlanResponse_GetStorageLevel_) isAnalyzePlanResponse_Res...
type AnalyzePlanResponse_JsonToDdl (line 898) | type AnalyzePlanResponse_JsonToDdl struct
method isAnalyzePlanResponse_Result (line 928) | func (*AnalyzePlanResponse_JsonToDdl) isAnalyzePlanResponse_Result() {}
type ExecutePlanRequest (line 931) | type ExecutePlanRequest struct
method Reset (line 973) | func (x *ExecutePlanRequest) Reset() {
method String (line 982) | func (x *ExecutePlanRequest) String() string {
method ProtoMessage (line 986) | func (*ExecutePlanRequest) ProtoMessage() {}
method ProtoReflect (line 988) | func (x *ExecutePlanRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 1001) | func (*ExecutePlanRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 1005) | func (x *ExecutePlanRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 1012) | func (x *ExecutePlanRequest) GetClientObservedServerSideSessionId() st...
method GetUserContext (line 1019) | func (x *ExecutePlanRequest) GetUserContext() *UserContext {
method GetOperationId (line 1026) | func (x *ExecutePlanRequest) GetOperationId() string {
method GetPlan (line 1033) | func (x *ExecutePlanRequest) GetPlan() *Plan {
method GetClientType (line 1040) | func (x *ExecutePlanRequest) GetClientType() string {
method GetRequestOptions (line 1047) | func (x *ExecutePlanRequest) GetRequestOptions() []*ExecutePlanRequest...
method GetTags (line 1054) | func (x *ExecutePlanRequest) GetTags() []string {
type ExecutePlanResponse (line 1064) | type ExecutePlanResponse struct
method Reset (line 1110) | func (x *ExecutePlanResponse) Reset() {
method String (line 1119) | func (x *ExecutePlanResponse) String() string {
method ProtoMessage (line 1123) | func (*ExecutePlanResponse) ProtoMessage() {}
method ProtoReflect (line 1125) | func (x *ExecutePlanResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 1138) | func (*ExecutePlanResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 1142) | func (x *ExecutePlanResponse) GetSessionId() string {
method GetServerSideSessionId (line 1149) | func (x *ExecutePlanResponse) GetServerSideSessionId() string {
method GetOperationId (line 1156) | func (x *ExecutePlanResponse) GetOperationId() string {
method GetResponseId (line 1163) | func (x *ExecutePlanResponse) GetResponseId() string {
method GetResponseType (line 1170) | func (m *ExecutePlanResponse) GetResponseType() isExecutePlanResponse_...
method GetArrowBatch (line 1177) | func (x *ExecutePlanResponse) GetArrowBatch() *ExecutePlanResponse_Arr...
method GetSqlCommandResult (line 1184) | func (x *ExecutePlanResponse) GetSqlCommandResult() *ExecutePlanRespon...
method GetWriteStreamOperationStartResult (line 1191) | func (x *ExecutePlanResponse) GetWriteStreamOperationStartResult() *Wr...
method GetStreamingQueryCommandResult (line 1198) | func (x *ExecutePlanResponse) GetStreamingQueryCommandResult() *Stream...
method GetGetResourcesCommandResult (line 1205) | func (x *ExecutePlanResponse) GetGetResourcesCommandResult() *GetResou...
method GetStreamingQueryManagerCommandResult (line 1212) | func (x *ExecutePlanResponse) GetStreamingQueryManagerCommandResult() ...
method GetStreamingQueryListenerEventsResult (line 1219) | func (x *ExecutePlanResponse) GetStreamingQueryListenerEventsResult() ...
method GetResultComplete (line 1226) | func (x *ExecutePlanResponse) GetResultComplete() *ExecutePlanResponse...
method GetCreateResourceProfileCommandResult (line 1233) | func (x *ExecutePlanResponse) GetCreateResourceProfileCommandResult() ...
method GetExecutionProgress (line 1240) | func (x *ExecutePlanResponse) GetExecutionProgress() *ExecutePlanRespo...
method GetCheckpointCommandResult (line 1247) | func (x *ExecutePlanResponse) GetCheckpointCommandResult() *Checkpoint...
method GetMlCommandResult (line 1254) | func (x *ExecutePlanResponse) GetMlCommandResult() *MlCommandResult {
method GetPipelineEventResult (line 1261) | func (x *ExecutePlanResponse) GetPipelineEventResult() *PipelineEventR...
method GetPipelineCommandResult (line 1268) | func (x *ExecutePlanResponse) GetPipelineCommandResult() *PipelineComm...
method GetExtension (line 1275) | func (x *ExecutePlanResponse) GetExtension() *anypb.Any {
method GetMetrics (line 1282) | func (x *ExecutePlanResponse) GetMetrics() *ExecutePlanResponse_Metrics {
method GetObservedMetrics (line 1289) | func (x *ExecutePlanResponse) GetObservedMetrics() []*ExecutePlanRespo...
method GetSchema (line 1296) | func (x *ExecutePlanResponse) GetSchema() *DataType {
type isExecutePlanResponse_ResponseType (line 1303) | type isExecutePlanResponse_ResponseType interface
type ExecutePlanResponse_ArrowBatch_ (line 1307) | type ExecutePlanResponse_ArrowBatch_ struct
method isExecutePlanResponse_ResponseType (line 1381) | func (*ExecutePlanResponse_ArrowBatch_) isExecutePlanResponse_Response...
type ExecutePlanResponse_SqlCommandResult_ (line 1311) | type ExecutePlanResponse_SqlCommandResult_ struct
method isExecutePlanResponse_ResponseType (line 1383) | func (*ExecutePlanResponse_SqlCommandResult_) isExecutePlanResponse_Re...
type ExecutePlanResponse_WriteStreamOperationStartResult (line 1316) | type ExecutePlanResponse_WriteStreamOperationStartResult struct
method isExecutePlanResponse_ResponseType (line 1385) | func (*ExecutePlanResponse_WriteStreamOperationStartResult) isExecuteP...
type ExecutePlanResponse_StreamingQueryCommandResult (line 1321) | type ExecutePlanResponse_StreamingQueryCommandResult struct
method isExecutePlanResponse_ResponseType (line 1387) | func (*ExecutePlanResponse_StreamingQueryCommandResult) isExecutePlanR...
type ExecutePlanResponse_GetResourcesCommandResult (line 1326) | type ExecutePlanResponse_GetResourcesCommandResult struct
method isExecutePlanResponse_ResponseType (line 1389) | func (*ExecutePlanResponse_GetResourcesCommandResult) isExecutePlanRes...
type ExecutePlanResponse_StreamingQueryManagerCommandResult (line 1331) | type ExecutePlanResponse_StreamingQueryManagerCommandResult struct
method isExecutePlanResponse_ResponseType (line 1391) | func (*ExecutePlanResponse_StreamingQueryManagerCommandResult) isExecu...
type ExecutePlanResponse_StreamingQueryListenerEventsResult (line 1336) | type ExecutePlanResponse_StreamingQueryListenerEventsResult struct
method isExecutePlanResponse_ResponseType (line 1393) | func (*ExecutePlanResponse_StreamingQueryListenerEventsResult) isExecu...
type ExecutePlanResponse_ResultComplete_ (line 1341) | type ExecutePlanResponse_ResultComplete_ struct
method isExecutePlanResponse_ResponseType (line 1395) | func (*ExecutePlanResponse_ResultComplete_) isExecutePlanResponse_Resp...
type ExecutePlanResponse_CreateResourceProfileCommandResult (line 1346) | type ExecutePlanResponse_CreateResourceProfileCommandResult struct
method isExecutePlanResponse_ResponseType (line 1397) | func (*ExecutePlanResponse_CreateResourceProfileCommandResult) isExecu...
type ExecutePlanResponse_ExecutionProgress_ (line 1351) | type ExecutePlanResponse_ExecutionProgress_ struct
method isExecutePlanResponse_ResponseType (line 1399) | func (*ExecutePlanResponse_ExecutionProgress_) isExecutePlanResponse_R...
type ExecutePlanResponse_CheckpointCommandResult (line 1356) | type ExecutePlanResponse_CheckpointCommandResult struct
method isExecutePlanResponse_ResponseType (line 1401) | func (*ExecutePlanResponse_CheckpointCommandResult) isExecutePlanRespo...
type ExecutePlanResponse_MlCommandResult (line 1361) | type ExecutePlanResponse_MlCommandResult struct
method isExecutePlanResponse_ResponseType (line 1403) | func (*ExecutePlanResponse_MlCommandResult) isExecutePlanResponse_Resp...
type ExecutePlanResponse_PipelineEventResult (line 1366) | type ExecutePlanResponse_PipelineEventResult struct
method isExecutePlanResponse_ResponseType (line 1405) | func (*ExecutePlanResponse_PipelineEventResult) isExecutePlanResponse_...
type ExecutePlanResponse_PipelineCommandResult (line 1371) | type ExecutePlanResponse_PipelineCommandResult struct
method isExecutePlanResponse_ResponseType (line 1407) | func (*ExecutePlanResponse_PipelineCommandResult) isExecutePlanRespons...
type ExecutePlanResponse_Extension (line 1376) | type ExecutePlanResponse_Extension struct
method isExecutePlanResponse_ResponseType (line 1409) | func (*ExecutePlanResponse_Extension) isExecutePlanResponse_ResponseTy...
type KeyValue (line 1412) | type KeyValue struct
method Reset (line 1423) | func (x *KeyValue) Reset() {
method String (line 1432) | func (x *KeyValue) String() string {
method ProtoMessage (line 1436) | func (*KeyValue) ProtoMessage() {}
method ProtoReflect (line 1438) | func (x *KeyValue) ProtoReflect() protoreflect.Message {
method Descriptor (line 1451) | func (*KeyValue) Descriptor() ([]byte, []int) {
method GetKey (line 1455) | func (x *KeyValue) GetKey() string {
method GetValue (line 1462) | func (x *KeyValue) GetValue() string {
type ConfigRequest (line 1470) | type ConfigRequest struct
method Reset (line 1497) | func (x *ConfigRequest) Reset() {
method String (line 1506) | func (x *ConfigRequest) String() string {
method ProtoMessage (line 1510) | func (*ConfigRequest) ProtoMessage() {}
method ProtoReflect (line 1512) | func (x *ConfigRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 1525) | func (*ConfigRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 1529) | func (x *ConfigRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 1536) | func (x *ConfigRequest) GetClientObservedServerSideSessionId() string {
method GetUserContext (line 1543) | func (x *ConfigRequest) GetUserContext() *UserContext {
method GetOperation (line 1550) | func (x *ConfigRequest) GetOperation() *ConfigRequest_Operation {
method GetClientType (line 1557) | func (x *ConfigRequest) GetClientType() string {
type ConfigResponse (line 1566) | type ConfigResponse struct
method Reset (line 1586) | func (x *ConfigResponse) Reset() {
method String (line 1595) | func (x *ConfigResponse) String() string {
method ProtoMessage (line 1599) | func (*ConfigResponse) ProtoMessage() {}
method ProtoReflect (line 1601) | func (x *ConfigResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 1614) | func (*ConfigResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 1618) | func (x *ConfigResponse) GetSessionId() string {
method GetServerSideSessionId (line 1625) | func (x *ConfigResponse) GetServerSideSessionId() string {
method GetPairs (line 1632) | func (x *ConfigResponse) GetPairs() []*KeyValue {
method GetWarnings (line 1639) | func (x *ConfigResponse) GetWarnings() []string {
type AddArtifactsRequest (line 1647) | type AddArtifactsRequest struct
method Reset (line 1680) | func (x *AddArtifactsRequest) Reset() {
method String (line 1689) | func (x *AddArtifactsRequest) String() string {
method ProtoMessage (line 1693) | func (*AddArtifactsRequest) ProtoMessage() {}
method ProtoReflect (line 1695) | func (x *AddArtifactsRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 1708) | func (*AddArtifactsRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 1712) | func (x *AddArtifactsRequest) GetSessionId() string {
method GetUserContext (line 1719) | func (x *AddArtifactsRequest) GetUserContext() *UserContext {
method GetClientObservedServerSideSessionId (line 1726) | func (x *AddArtifactsRequest) GetClientObservedServerSideSessionId() s...
method GetClientType (line 1733) | func (x *AddArtifactsRequest) GetClientType() string {
method GetPayload (line 1740) | func (m *AddArtifactsRequest) GetPayload() isAddArtifactsRequest_Paylo...
method GetBatch (line 1747) | func (x *AddArtifactsRequest) GetBatch() *AddArtifactsRequest_Batch {
method GetBeginChunk (line 1754) | func (x *AddArtifactsRequest) GetBeginChunk() *AddArtifactsRequest_Beg...
method GetChunk (line 1761) | func (x *AddArtifactsRequest) GetChunk() *AddArtifactsRequest_Artifact...
type isAddArtifactsRequest_Payload (line 1768) | type isAddArtifactsRequest_Payload interface
type AddArtifactsRequest_Batch_ (line 1772) | type AddArtifactsRequest_Batch_ struct
method isAddArtifactsRequest_Payload (line 1789) | func (*AddArtifactsRequest_Batch_) isAddArtifactsRequest_Payload() {}
type AddArtifactsRequest_BeginChunk (line 1776) | type AddArtifactsRequest_BeginChunk struct
method isAddArtifactsRequest_Payload (line 1791) | func (*AddArtifactsRequest_BeginChunk) isAddArtifactsRequest_Payload() {}
type AddArtifactsRequest_Chunk (line 1783) | type AddArtifactsRequest_Chunk struct
method isAddArtifactsRequest_Payload (line 1793) | func (*AddArtifactsRequest_Chunk) isAddArtifactsRequest_Payload() {}
type AddArtifactsResponse (line 1798) | type AddArtifactsResponse struct
method Reset (line 1812) | func (x *AddArtifactsResponse) Reset() {
method String (line 1821) | func (x *AddArtifactsResponse) String() string {
method ProtoMessage (line 1825) | func (*AddArtifactsResponse) ProtoMessage() {}
method ProtoReflect (line 1827) | func (x *AddArtifactsResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 1840) | func (*AddArtifactsResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 1844) | func (x *AddArtifactsResponse) GetSessionId() string {
method GetServerSideSessionId (line 1851) | func (x *AddArtifactsResponse) GetServerSideSessionId() string {
method GetArtifacts (line 1858) | func (x *AddArtifactsResponse) GetArtifacts() []*AddArtifactsResponse_...
type ArtifactStatusesRequest (line 1866) | type ArtifactStatusesRequest struct
method Reset (line 1898) | func (x *ArtifactStatusesRequest) Reset() {
method String (line 1907) | func (x *ArtifactStatusesRequest) String() string {
method ProtoMessage (line 1911) | func (*ArtifactStatusesRequest) ProtoMessage() {}
method ProtoReflect (line 1913) | func (x *ArtifactStatusesRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 1926) | func (*ArtifactStatusesRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 1930) | func (x *ArtifactStatusesRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 1937) | func (x *ArtifactStatusesRequest) GetClientObservedServerSideSessionId...
method GetUserContext (line 1944) | func (x *ArtifactStatusesRequest) GetUserContext() *UserContext {
method GetClientType (line 1951) | func (x *ArtifactStatusesRequest) GetClientType() string {
method GetNames (line 1958) | func (x *ArtifactStatusesRequest) GetNames() []string {
type ArtifactStatusesResponse (line 1967) | type ArtifactStatusesResponse struct
method Reset (line 1981) | func (x *ArtifactStatusesResponse) Reset() {
method String (line 1990) | func (x *ArtifactStatusesResponse) String() string {
method ProtoMessage (line 1994) | func (*ArtifactStatusesResponse) ProtoMessage() {}
method ProtoReflect (line 1996) | func (x *ArtifactStatusesResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 2009) | func (*ArtifactStatusesResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 2013) | func (x *ArtifactStatusesResponse) GetSessionId() string {
method GetServerSideSessionId (line 2020) | func (x *ArtifactStatusesResponse) GetServerSideSessionId() string {
method GetStatuses (line 2027) | func (x *ArtifactStatusesResponse) GetStatuses() map[string]*ArtifactS...
type InterruptRequest (line 2034) | type InterruptRequest struct
method Reset (line 2066) | func (x *InterruptRequest) Reset() {
method String (line 2075) | func (x *InterruptRequest) String() string {
method ProtoMessage (line 2079) | func (*InterruptRequest) ProtoMessage() {}
method ProtoReflect (line 2081) | func (x *InterruptRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 2094) | func (*InterruptRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 2098) | func (x *InterruptRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 2105) | func (x *InterruptRequest) GetClientObservedServerSideSessionId() stri...
method GetUserContext (line 2112) | func (x *InterruptRequest) GetUserContext() *UserContext {
method GetClientType (line 2119) | func (x *InterruptRequest) GetClientType() string {
method GetInterruptType (line 2126) | func (x *InterruptRequest) GetInterruptType() InterruptRequest_Interru...
method GetInterrupt (line 2133) | func (m *InterruptRequest) GetInterrupt() isInterruptRequest_Interrupt {
method GetOperationTag (line 2140) | func (x *InterruptRequest) GetOperationTag() string {
method GetOperationId (line 2147) | func (x *InterruptRequest) GetOperationId() string {
type isInterruptRequest_Interrupt (line 2154) | type isInterruptRequest_Interrupt interface
type InterruptRequest_OperationTag (line 2158) | type InterruptRequest_OperationTag struct
method isInterruptRequest_Interrupt (line 2168) | func (*InterruptRequest_OperationTag) isInterruptRequest_Interrupt() {}
type InterruptRequest_OperationId (line 2163) | type InterruptRequest_OperationId struct
method isInterruptRequest_Interrupt (line 2170) | func (*InterruptRequest_OperationId) isInterruptRequest_Interrupt() {}
type InterruptResponse (line 2173) | type InterruptResponse struct
method Reset (line 2187) | func (x *InterruptResponse) Reset() {
method String (line 2196) | func (x *InterruptResponse) String() string {
method ProtoMessage (line 2200) | func (*InterruptResponse) ProtoMessage() {}
method ProtoReflect (line 2202) | func (x *InterruptResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 2215) | func (*InterruptResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 2219) | func (x *InterruptResponse) GetSessionId() string {
method GetServerSideSessionId (line 2226) | func (x *InterruptResponse) GetServerSideSessionId() string {
method GetInterruptedIds (line 2233) | func (x *InterruptResponse) GetInterruptedIds() []string {
type ReattachOptions (line 2240) | type ReattachOptions struct
method Reset (line 2256) | func (x *ReattachOptions) Reset() {
method String (line 2265) | func (x *ReattachOptions) String() string {
method ProtoMessage (line 2269) | func (*ReattachOptions) ProtoMessage() {}
method ProtoReflect (line 2271) | func (x *ReattachOptions) ProtoReflect() protoreflect.Message {
method Descriptor (line 2284) | func (*ReattachOptions) Descriptor() ([]byte, []int) {
method GetReattachable (line 2288) | func (x *ReattachOptions) GetReattachable() bool {
type ReattachExecuteRequest (line 2295) | type ReattachExecuteRequest struct
method Reset (line 2334) | func (x *ReattachExecuteRequest) Reset() {
method String (line 2343) | func (x *ReattachExecuteRequest) String() string {
method ProtoMessage (line 2347) | func (*ReattachExecuteRequest) ProtoMessage() {}
method ProtoReflect (line 2349) | func (x *ReattachExecuteRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 2362) | func (*ReattachExecuteRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 2366) | func (x *ReattachExecuteRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 2373) | func (x *ReattachExecuteRequest) GetClientObservedServerSideSessionId(...
method GetUserContext (line 2380) | func (x *ReattachExecuteRequest) GetUserContext() *UserContext {
method GetOperationId (line 2387) | func (x *ReattachExecuteRequest) GetOperationId() string {
method GetClientType (line 2394) | func (x *ReattachExecuteRequest) GetClientType() string {
method GetLastResponseId (line 2401) | func (x *ReattachExecuteRequest) GetLastResponseId() string {
type ReleaseExecuteRequest (line 2408) | type ReleaseExecuteRequest struct
method Reset (line 2443) | func (x *ReleaseExecuteRequest) Reset() {
method String (line 2452) | func (x *ReleaseExecuteRequest) String() string {
method ProtoMessage (line 2456) | func (*ReleaseExecuteRequest) ProtoMessage() {}
method ProtoReflect (line 2458) | func (x *ReleaseExecuteRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 2471) | func (*ReleaseExecuteRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 2475) | func (x *ReleaseExecuteRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 2482) | func (x *ReleaseExecuteRequest) GetClientObservedServerSideSessionId()...
method GetUserContext (line 2489) | func (x *ReleaseExecuteRequest) GetUserContext() *UserContext {
method GetOperationId (line 2496) | func (x *ReleaseExecuteRequest) GetOperationId() string {
method GetClientType (line 2503) | func (x *ReleaseExecuteRequest) GetClientType() string {
method GetRelease (line 2510) | func (m *ReleaseExecuteRequest) GetRelease() isReleaseExecuteRequest_R...
method GetReleaseAll (line 2517) | func (x *ReleaseExecuteRequest) GetReleaseAll() *ReleaseExecuteRequest...
method GetReleaseUntil (line 2524) | func (x *ReleaseExecuteRequest) GetReleaseUntil() *ReleaseExecuteReque...
type isReleaseExecuteRequest_Release (line 2531) | type isReleaseExecuteRequest_Release interface
type ReleaseExecuteRequest_ReleaseAll_ (line 2535) | type ReleaseExecuteRequest_ReleaseAll_ struct
method isReleaseExecuteRequest_Release (line 2543) | func (*ReleaseExecuteRequest_ReleaseAll_) isReleaseExecuteRequest_Rele...
type ReleaseExecuteRequest_ReleaseUntil_ (line 2539) | type ReleaseExecuteRequest_ReleaseUntil_ struct
method isReleaseExecuteRequest_Release (line 2545) | func (*ReleaseExecuteRequest_ReleaseUntil_) isReleaseExecuteRequest_Re...
type ReleaseExecuteResponse (line 2548) | type ReleaseExecuteResponse struct
method Reset (line 2564) | func (x *ReleaseExecuteResponse) Reset() {
method String (line 2573) | func (x *ReleaseExecuteResponse) String() string {
method ProtoMessage (line 2577) | func (*ReleaseExecuteResponse) ProtoMessage() {}
method ProtoReflect (line 2579) | func (x *ReleaseExecuteResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 2592) | func (*ReleaseExecuteResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 2596) | func (x *ReleaseExecuteResponse) GetSessionId() string {
method GetServerSideSessionId (line 2603) | func (x *ReleaseExecuteResponse) GetServerSideSessionId() string {
method GetOperationId (line 2610) | func (x *ReleaseExecuteResponse) GetOperationId() string {
type ReleaseSessionRequest (line 2617) | type ReleaseSessionRequest struct
method Reset (line 2651) | func (x *ReleaseSessionRequest) Reset() {
method String (line 2660) | func (x *ReleaseSessionRequest) String() string {
method ProtoMessage (line 2664) | func (*ReleaseSessionRequest) ProtoMessage() {}
method ProtoReflect (line 2666) | func (x *ReleaseSessionRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 2679) | func (*ReleaseSessionRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 2683) | func (x *ReleaseSessionRequest) GetSessionId() string {
method GetUserContext (line 2690) | func (x *ReleaseSessionRequest) GetUserContext() *UserContext {
method GetClientType (line 2697) | func (x *ReleaseSessionRequest) GetClientType() string {
method GetAllowReconnect (line 2704) | func (x *ReleaseSessionRequest) GetAllowReconnect() bool {
type ReleaseSessionResponse (line 2712) | type ReleaseSessionResponse struct
method Reset (line 2724) | func (x *ReleaseSessionResponse) Reset() {
method String (line 2733) | func (x *ReleaseSessionResponse) String() string {
method ProtoMessage (line 2737) | func (*ReleaseSessionResponse) ProtoMessage() {}
method ProtoReflect (line 2739) | func (x *ReleaseSessionResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 2752) | func (*ReleaseSessionResponse) Descriptor() ([]byte, []int) {
method GetSessionId (line 2756) | func (x *ReleaseSessionResponse) GetSessionId() string {
method GetServerSideSessionId (line 2763) | func (x *ReleaseSessionResponse) GetServerSideSessionId() string {
type FetchErrorDetailsRequest (line 2770) | type FetchErrorDetailsRequest struct
method Reset (line 2795) | func (x *FetchErrorDetailsRequest) Reset() {
method String (line 2804) | func (x *FetchErrorDetailsRequest) String() string {
method ProtoMessage (line 2808) | func (*FetchErrorDetailsRequest) ProtoMessage() {}
method ProtoReflect (line 2810) | func (x *FetchErrorDetailsRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 2823) | func (*FetchErrorDetailsRequest) Descriptor() ([]byte, []int) {
method GetSessionId (line 2827) | func (x *FetchErrorDetailsRequest) GetSessionId() string {
method GetClientObservedServerSideSessionId (line 2834) | func (x *FetchErrorDetailsRequest) GetClientObservedServerSideSessionI...
method GetUserContext (line 2841) | func (x *FetchErrorDetailsRequest) GetUserContext() *UserContext {
method GetErrorId (line 2848) | func (x *FetchErrorDetailsRequest) GetErrorId() string {
method GetClientType (line 2855) | func (x *FetchErrorDetailsRequest) GetClientType() string {
type FetchErrorDetailsResponse (line 2863) | type FetchErrorDetailsResponse struct
method Reset (line 2878) | func (x *FetchErrorDetailsResponse) Reset() {
method String (line 2887) | func (x *FetchErrorDetailsResponse) String() string {
method ProtoMessage (line 2891) | func (*FetchErrorDetailsResponse) ProtoMessage() {}
method ProtoReflect (line 2893) | func (x *FetchErrorDetailsResponse) ProtoReflect() protoreflect.Message {
method Descriptor (line 2906) | func (*FetchErrorDetailsResponse) Descriptor() ([]byte, []int) {
method GetServerSideSessionId (line 2910) | func (x *FetchErrorDetailsResponse) GetServerSideSessionId() string {
method GetSessionId (line 2917) | func (x *FetchErrorDetailsResponse) GetSessionId() string {
method GetRootErrorIdx (line 2924) | func (x *FetchErrorDetailsResponse) GetRootErrorIdx() int32 {
method GetErrors (line 2931) | func (x *FetchErrorDetailsResponse) GetErrors() []*FetchErrorDetailsRe...
type CheckpointCommandResult (line 2938) | type CheckpointCommandResult struct
method Reset (line 2947) | func (x *CheckpointCommandResult) Reset() {
method String (line 2956) | func (x *CheckpointCommandResult) String() string {
method ProtoMessage (line 2960) | func (*CheckpointCommandResult) ProtoMessage() {}
method ProtoReflect (line 2962) | func (x *CheckpointCommandResult) ProtoReflect() protoreflect.Message {
method Descriptor (line 2975) | func (*CheckpointCommandResult) Descriptor() ([]byte, []int) {
method GetRelation (line 2979) | func (x *CheckpointCommandResult) GetRelation() *CachedRemoteRelation {
type AnalyzePlanRequest_Schema (line 2986) | type AnalyzePlanRequest_Schema struct
method Reset (line 2995) | func (x *AnalyzePlanRequest_Schema) Reset() {
method String (line 3004) | func (x *AnalyzePlanRequest_Schema) String() string {
method ProtoMessage (line 3008) | func (*AnalyzePlanRequest_Schema) ProtoMessage() {}
method ProtoReflect (line 3010) | func (x *AnalyzePlanRequest_Schema) ProtoReflect() protoreflect.Message {
method Descriptor (line 3023) | func (*AnalyzePlanRequest_Schema) Descriptor() ([]byte, []int) {
method GetPlan (line 3027) | func (x *AnalyzePlanRequest_Schema) GetPlan() *Plan {
type AnalyzePlanRequest_Explain (line 3035) | type AnalyzePlanRequest_Explain struct
method Reset (line 3046) | func (x *AnalyzePlanRequest_Explain) Reset() {
method String (line 3055) | func (x *AnalyzePlanRequest_Explain) String() string {
method ProtoMessage (line 3059) | func (*AnalyzePlanRequest_Explain) ProtoMessage() {}
method ProtoReflect (line 3061) | func (x *AnalyzePlanRequest_Explain) ProtoReflect() protoreflect.Messa...
method Descriptor (line 3074) | func (*AnalyzePlanRequest_Explain) Descriptor() ([]byte, []int) {
method GetPlan (line 3078) | func (x *AnalyzePlanRequest_Explain) GetPlan() *Plan {
method GetExplainMode (line 3085) | func (x *AnalyzePlanRequest_Explain) GetExplainMode() AnalyzePlanReque...
type AnalyzePlanRequest_TreeString (line 3092) | type AnalyzePlanRequest_TreeString struct
method Reset (line 3103) | func (x *AnalyzePlanRequest_TreeString) Reset() {
method String (line 3112) | func (x *AnalyzePlanRequest_TreeString) String() string {
method ProtoMessage (line 3116) | func (*AnalyzePlanRequest_TreeString) ProtoMessage() {}
method ProtoReflect (line 3118) | func (x *AnalyzePlanRequest_TreeString) ProtoReflect() protoreflect.Me...
method Descriptor (line 3131) | func (*AnalyzePlanRequest_TreeString) Descriptor() ([]byte, []int) {
method GetPlan (line 3135) | func (x *AnalyzePlanRequest_TreeString) GetPlan() *Plan {
method GetLevel (line 3142) | func (x *AnalyzePlanRequest_TreeString) GetLevel() int32 {
type AnalyzePlanRequest_IsLocal (line 3149) | type AnalyzePlanRequest_IsLocal struct
method Reset (line 3158) | func (x *AnalyzePlanRequest_IsLocal) Reset() {
method String (line 3167) | func (x *AnalyzePlanRequest_IsLocal) String() string {
method ProtoMessage (line 3171) | func (*AnalyzePlanRequest_IsLocal) ProtoMessage() {}
method ProtoReflect (line 3173) | func (x *AnalyzePlanRequest_IsLocal) ProtoReflect() protoreflect.Messa...
method Descriptor (line 3186) | func (*AnalyzePlanRequest_IsLocal) Descriptor() ([]byte, []int) {
method GetPlan (line 3190) | func (x *AnalyzePlanRequest_IsLocal) GetPlan() *Plan {
type AnalyzePlanRequest_IsStreaming (line 3197) | type AnalyzePlanRequest_IsStreaming struct
method Reset (line 3206) | func (x *AnalyzePlanRequest_IsStreaming) Reset() {
method String (line 3215) | func (x *AnalyzePlanRequest_IsStreaming) String() string {
method ProtoMessage (line 3219) | func (*AnalyzePlanRequest_IsStreaming) ProtoMessage() {}
method ProtoReflect (line 3221) | func (x *AnalyzePlanRequest_IsStreaming) ProtoReflect() protoreflect.M...
method Descriptor (line 3234) | func (*AnalyzePlanRequest_IsStreaming) Descriptor() ([]byte, []int) {
method GetPlan (line 3238) | func (x *AnalyzePlanRequest_IsStreaming) GetPlan() *Plan {
type AnalyzePlanRequest_InputFiles (line 3245) | type AnalyzePlanRequest_InputFiles struct
method Reset (line 3254) | func (x *AnalyzePlanRequest_InputFiles) Reset() {
method String (line 3263) | func (x *AnalyzePlanRequest_InputFiles) String() string {
method ProtoMessage (line 3267) | func (*AnalyzePlanRequest_InputFiles) ProtoMessage() {}
method ProtoReflect (line 3269) | func (x *AnalyzePlanRequest_InputFiles) ProtoReflect() protoreflect.Me...
method Descriptor (line 3282) | func (*AnalyzePlanRequest_InputFiles) Descriptor() ([]byte, []int) {
method GetPlan (line 3286) | func (x *AnalyzePlanRequest_InputFiles) GetPlan() *Plan {
type AnalyzePlanRequest_SparkVersion (line 3293) | type AnalyzePlanRequest_SparkVersion struct
method Reset (line 3299) | func (x *AnalyzePlanRequest_SparkVersion) Reset() {
method String (line 3308) | func (x *AnalyzePlanRequest_SparkVersion) String() string {
method ProtoMessage (line 3312) | func (*AnalyzePlanRequest_SparkVersion) ProtoMessage() {}
method ProtoReflect (line 3314) | func (x *AnalyzePlanRequest_SparkVersion) ProtoReflect() protoreflect....
method Descriptor (line 3327) | func (*AnalyzePlanRequest_SparkVersion) Descriptor() ([]byte, []int) {
type AnalyzePlanRequest_DDLParse (line 3331) | type AnalyzePlanRequest_DDLParse struct
method Reset (line 3340) | func (x *AnalyzePlanRequest_DDLParse) Reset() {
method String (line 3349) | func (x *AnalyzePlanRequest_DDLParse) String() string {
method ProtoMessage (line 3353) | func (*AnalyzePlanRequest_DDLParse) ProtoMessage() {}
method ProtoReflect (line 3355) | func (x *AnalyzePlanRequest_DDLParse) ProtoReflect() protoreflect.Mess...
method Descriptor (line 3368) | func (*AnalyzePlanRequest_DDLParse) Descriptor() ([]byte, []int) {
method GetDdlString (line 3372) | func (x *AnalyzePlanRequest_DDLParse) GetDdlString() string {
type AnalyzePlanRequest_SameSemantics (line 3380) | type AnalyzePlanRequest_SameSemantics struct
method Reset (line 3391) | func (x *AnalyzePlanRequest_SameSemantics) Reset() {
method String (line 3400) | func (x *AnalyzePlanRequest_SameSemantics) String() string {
method ProtoMessage (line 3404) | func (*AnalyzePlanRequest_SameSemantics) ProtoMessage() {}
method ProtoReflect (line 3406) | func (x *AnalyzePlanRequest_SameSemantics) ProtoReflect() protoreflect...
method Descriptor (line 3419) | func (*AnalyzePlanRequest_SameSemantics) Descriptor() ([]byte, []int) {
method GetTargetPlan (line 3423) | func (x *AnalyzePlanRequest_SameSemantics) GetTargetPlan() *Plan {
method GetOtherPlan (line 3430) | func (x *AnalyzePlanRequest_SameSemantics) GetOtherPlan() *Plan {
type AnalyzePlanRequest_SemanticHash (line 3437) | type AnalyzePlanRequest_SemanticHash struct
method Reset (line 3446) | func (x *AnalyzePlanRequest_SemanticHash) Reset() {
method String (line 3455) | func (x *AnalyzePlanRequest_SemanticHash) String() string {
method ProtoMessage (line 3459) | func (*AnalyzePlanRequest_SemanticHash) ProtoMessage() {}
method ProtoReflect (line 3461) | func (x *AnalyzePlanRequest_SemanticHash) ProtoReflect() protoreflect....
method Descriptor (line 3474) | func (*AnalyzePlanRequest_SemanticHash) Descriptor() ([]byte, []int) {
method GetPlan (line 3478) | func (x *AnalyzePlanRequest_SemanticHash) GetPlan() *Plan {
type AnalyzePlanRequest_Persist (line 3485) | type AnalyzePlanRequest_Persist struct
method Reset (line 3496) | func (x *AnalyzePlanRequest_Persist) Reset() {
method String (line 3505) | func (x *AnalyzePlanRequest_Persist) String() string {
method ProtoMessage (line 3509) | func (*AnalyzePlanRequest_Persist) ProtoMessage() {}
method ProtoReflect (line 3511) | func (x *AnalyzePlanRequest_Persist) ProtoReflect() protoreflect.Messa...
method Descriptor (line 3524) | func (*AnalyzePlanRequest_Persist) Descriptor() ([]byte, []int) {
method GetRelation (line 3528) | func (x *AnalyzePlanRequest_Persist) GetRelation() *Relation {
method GetStorageLevel (line 3535) | func (x *AnalyzePlanRequest_Persist) GetStorageLevel() *StorageLevel {
type AnalyzePlanRequest_Unpersist (line 3542) | type AnalyzePlanRequest_Unpersist struct
method Reset (line 3553) | func (x *AnalyzePlanRequest_Unpersist) Reset() {
method String (line 3562) | func (x *AnalyzePlanRequest_Unpersist) String() string {
method ProtoMessage (line 3566) | func (*AnalyzePlanRequest_Unpersist) ProtoMessage() {}
method ProtoReflect (line 3568) | func (x *AnalyzePlanRequest_Unpersist) ProtoReflect() protoreflect.Mes...
method Descriptor (line 3581) | func (*AnalyzePlanRequest_Unpersist) Descriptor() ([]byte, []int) {
method GetRelation (line 3585) | func (x *AnalyzePlanRequest_Unpersist) GetRelation() *Relation {
method GetBlocking (line 3592) | func (x *AnalyzePlanRequest_Unpersist) GetBlocking() bool {
type AnalyzePlanRequest_GetStorageLevel (line 3599) | type AnalyzePlanRequest_GetStorageLevel struct
method Reset (line 3608) | func (x *AnalyzePlanRequest_GetStorageLevel) Reset() {
method String (line 3617) | func (x *AnalyzePlanRequest_GetStorageLevel) String() string {
method ProtoMessage (line 3621) | func (*AnalyzePlanRequest_GetStorageLevel) ProtoMessage() {}
method ProtoReflect (line 3623) | func (x *AnalyzePlanRequest_GetStorageLevel) ProtoReflect() protorefle...
method Descriptor (line 3636) | func (*AnalyzePlanRequest_GetStorageLevel) Descriptor() ([]byte, []int) {
method GetRelation (line 3640) | func (x *AnalyzePlanRequest_GetStorageLevel) GetRelation() *Relation {
type AnalyzePlanRequest_JsonToDDL (line 3647) | type AnalyzePlanRequest_JsonToDDL struct
method Reset (line 3656) | func (x *AnalyzePlanRequest_JsonToDDL) Reset() {
method String (line 3665) | func (x *AnalyzePlanRequest_JsonToDDL) String() string {
method ProtoMessage (line 3669) | func (*AnalyzePlanRequest_JsonToDDL) ProtoMessage() {}
method ProtoReflect (line 3671) | func (x *AnalyzePlanRequest_JsonToDDL) ProtoReflect() protoreflect.Mes...
method Descriptor (line 3684) | func (*AnalyzePlanRequest_JsonToDDL) Descriptor() ([]byte, []int) {
method GetJsonString (line 3688) | func (x *AnalyzePlanRequest_JsonToDDL) GetJsonString() string {
type AnalyzePlanResponse_Schema (line 3695) | type AnalyzePlanResponse_Schema struct
method Reset (line 3703) | func (x *AnalyzePlanResponse_Schema) Reset() {
method String (line 3712) | func (x *AnalyzePlanResponse_Schema) String() string {
method ProtoMessage (line 3716) | func (*AnalyzePlanResponse_Schema) ProtoMessage() {}
method ProtoReflect (line 3718) | func (x *AnalyzePlanResponse_Schema) ProtoReflect() protoreflect.Messa...
method Descriptor (line 3731) | func (*AnalyzePlanResponse_Schema) Descriptor() ([]byte, []int) {
method GetSchema (line 3735) | func (x *AnalyzePlanResponse_Schema) GetSchema() *DataType {
type AnalyzePlanResponse_Explain (line 3742) | type AnalyzePlanResponse_Explain struct
method Reset (line 3750) | func (x *AnalyzePlanResponse_Explain) Reset() {
method String (line 3759) | func (x *AnalyzePlanResponse_Explain) String() string {
method ProtoMessage (line 3763) | func (*AnalyzePlanResponse_Explain) ProtoMessage() {}
method ProtoReflect (line 3765) | func (x *AnalyzePlanResponse_Explain) ProtoReflect() protoreflect.Mess...
method Descriptor (line 3778) | func (*AnalyzePlanResponse_Explain) Descriptor() ([]byte, []int) {
method GetExplainString (line 3782) | func (x *AnalyzePlanResponse_Explain) GetExplainString() string {
type AnalyzePlanResponse_TreeString (line 3789) | type AnalyzePlanResponse_TreeString struct
method Reset (line 3797) | func (x *AnalyzePlanResponse_TreeString) Reset() {
method String (line 3806) | func (x *AnalyzePlanResponse_TreeString) String() string {
method ProtoMessage (line 3810) | func (*AnalyzePlanResponse_TreeString) ProtoMessage() {}
method ProtoReflect (line 3812) | func (x *AnalyzePlanResponse_TreeString) ProtoReflect() protoreflect.M...
method Descriptor (line 3825) | func (*AnalyzePlanResponse_TreeString) Descriptor() ([]byte, []int) {
method GetTreeString (line 3829) | func (x *AnalyzePlanResponse_TreeString) GetTreeString() string {
type AnalyzePlanResponse_IsLocal (line 3836) | type AnalyzePlanResponse_IsLocal struct
method Reset (line 3844) | func (x *AnalyzePlanResponse_IsLocal) Reset() {
method String (line 3853) | func (x *AnalyzePlanResponse_IsLocal) String() string {
method ProtoMessage (line 3857) | func (*AnalyzePlanResponse_IsLocal) ProtoMessage() {}
method ProtoReflect (line 3859) | func (x *AnalyzePlanResponse_IsLocal) ProtoReflect() protoreflect.Mess...
method Descriptor (line 3872) | func (*AnalyzePlanResponse_IsLocal) Descriptor() ([]byte, []int) {
method GetIsLocal (line 3876) | func (x *AnalyzePlanResponse_IsLocal) GetIsLocal() bool {
type AnalyzePlanResponse_IsStreaming (line 3883) | type AnalyzePlanResponse_IsStreaming struct
method Reset (line 3891) | func (x *AnalyzePlanResponse_IsStreaming) Reset() {
method String (line 3900) | func (x *AnalyzePlanResponse_IsStreaming) String() string {
method ProtoMessage (line 3904) | func (*AnalyzePlanResponse_IsStreaming) ProtoMessage() {}
method ProtoReflect (line 3906) | func (x *AnalyzePlanResponse_IsStreaming) ProtoReflect() protoreflect....
method Descriptor (line 3919) | func (*AnalyzePlanResponse_IsStreaming) Descriptor() ([]byte, []int) {
method GetIsStreaming (line 3923) | func (x *AnalyzePlanResponse_IsStreaming) GetIsStreaming() bool {
type AnalyzePlanResponse_InputFiles (line 3930) | type AnalyzePlanResponse_InputFiles struct
method Reset (line 3939) | func (x *AnalyzePlanResponse_InputFiles) Reset() {
method String (line 3948) | func (x *AnalyzePlanResponse_InputFiles) String() string {
method ProtoMessage (line 3952) | func (*AnalyzePlanResponse_InputFiles) ProtoMessage() {}
method ProtoReflect (line 3954) | func (x *AnalyzePlanResponse_InputFiles) ProtoReflect() protoreflect.M...
method Descriptor (line 3967) | func (*AnalyzePlanResponse_InputFiles) Descriptor() ([]byte, []int) {
method GetFiles (line 3971) | func (x *AnalyzePlanResponse_InputFiles) GetFiles() []string {
type AnalyzePlanResponse_SparkVersion (line 3978) | type AnalyzePlanResponse_SparkVersion struct
method Reset (line 3986) | func (x *AnalyzePlanResponse_SparkVersion) Reset() {
method String (line 3995) | func (x *AnalyzePlanResponse_SparkVersion) String() string {
method ProtoMessage (line 3999) | func (*AnalyzePlanResponse_SparkVersion) ProtoMessage() {}
method ProtoReflect (line 4001) | func (x *AnalyzePlanResponse_SparkVersion) ProtoReflect() protoreflect...
method Descriptor (line 4014) | func (*AnalyzePlanResponse_SparkVersion) Descriptor() ([]byte, []int) {
method GetVersion (line 4018) | func (x *AnalyzePlanResponse_SparkVersion) GetVersion() string {
type AnalyzePlanResponse_DDLParse (line 4025) | type AnalyzePlanResponse_DDLParse struct
method Reset (line 4033) | func (x *AnalyzePlanResponse_DDLParse) Reset() {
method String (line 4042) | func (x *AnalyzePlanResponse_DDLParse) String() string {
method ProtoMessage (line 4046) | func (*AnalyzePlanResponse_DDLParse) ProtoMessage() {}
method ProtoReflect (line 4048) | func (x *AnalyzePlanResponse_DDLParse) ProtoReflect() protoreflect.Mes...
method Descriptor (line 4061) | func (*AnalyzePlanResponse_DDLParse) Descriptor() ([]byte, []int) {
method GetParsed (line 4065) | func (x *AnalyzePlanResponse_DDLParse) GetParsed() *DataType {
type AnalyzePlanResponse_SameSemantics (line 4072) | type AnalyzePlanResponse_SameSemantics struct
method Reset (line 4080) | func (x *AnalyzePlanResponse_SameSemantics) Reset() {
method String (line 4089) | func (x *AnalyzePlanResponse_SameSemantics) String() string {
method ProtoMessage (line 4093) | func (*AnalyzePlanResponse_SameSemantics) ProtoMessage() {}
method ProtoReflect (line 4095) | func (x *AnalyzePlanResponse_SameSemantics) ProtoReflect() protoreflec...
method Descriptor (line 4108) | func (*AnalyzePlanResponse_SameSemantics) Descriptor() ([]byte, []int) {
method GetResult (line 4112) | func (x *AnalyzePlanResponse_SameSemantics) GetResult() bool {
type AnalyzePlanResponse_SemanticHash (line 4119) | type AnalyzePlanResponse_SemanticHash struct
method Reset (line 4127) | func (x *AnalyzePlanResponse_SemanticHash) Reset() {
method String (line 4136) | func (x *AnalyzePlanResponse_SemanticHash) String() string {
method ProtoMessage (line 4140) | func (*AnalyzePlanResponse_SemanticHash) ProtoMessage() {}
method ProtoReflect (line 4142) | func (x *AnalyzePlanResponse_SemanticHash) ProtoReflect() protoreflect...
method Descriptor (line 4155) | func (*AnalyzePlanResponse_SemanticHash) Descriptor() ([]byte, []int) {
method GetResult (line 4159) | func (x *AnalyzePlanResponse_SemanticHash) GetResult() int32 {
type AnalyzePlanResponse_Persist (line 4166) | type AnalyzePlanResponse_Persist struct
method Reset (line 4172) | func (x *AnalyzePlanResponse_Persist) Reset() {
method String (line 4181) | func (x *AnalyzePlanResponse_Persist) String() string {
method ProtoMessage (line 4185) | func (*AnalyzePlanResponse_Persist) ProtoMessage() {}
method ProtoReflect (line 4187) | func (x *AnalyzePlanResponse_Persist) ProtoReflect() protoreflect.Mess...
method Descriptor (line 4200) | func (*AnalyzePlanResponse_Persist) Descriptor() ([]byte, []int) {
type AnalyzePlanResponse_Unpersist (line 4204) | type AnalyzePlanResponse_Unpersist struct
method Reset (line 4210) | func (x *AnalyzePlanResponse_Unpersist) Reset() {
method String (line 4219) | func (x *AnalyzePlanResponse_Unpersist) String() string {
method ProtoMessage (line 4223) | func (*AnalyzePlanResponse_Unpersist) ProtoMessage() {}
method ProtoReflect (line 4225) | func (x *AnalyzePlanResponse_Unpersist) ProtoReflect() protoreflect.Me...
method Descriptor (line 4238) | func (*AnalyzePlanResponse_Unpersist) Descriptor() ([]byte, []int) {
type AnalyzePlanResponse_GetStorageLevel (line 4242) | type AnalyzePlanResponse_GetStorageLevel struct
method Reset (line 4251) | func (x *AnalyzePlanResponse_GetStorageLevel) Reset() {
method String (line 4260) | func (x *AnalyzePlanResponse_GetStorageLevel) String() string {
method ProtoMessage (line 4264) | func (*AnalyzePlanResponse_GetStorageLevel) ProtoMessage() {}
method ProtoReflect (line 4266) | func (x *AnalyzePlanResponse_GetStorageLevel) ProtoReflect() protorefl...
method Descriptor (line 4279) | func (*AnalyzePlanResponse_GetStorageLevel) Descriptor() ([]byte, []in...
method GetStorageLevel (line 4283) | func (x *AnalyzePlanResponse_GetStorageLevel) GetStorageLevel() *Stora...
type AnalyzePlanResponse_JsonToDDL (line 4290) | type AnalyzePlanResponse_JsonToDDL struct
method Reset (line 4298) | func (x *AnalyzePlanResponse_JsonToDDL) Reset() {
method String (line 4307) | func (x *AnalyzePlanResponse_JsonToDDL) String() string {
method ProtoMessage (line 4311) | func (*AnalyzePlanResponse_JsonToDDL) ProtoMessage() {}
method ProtoReflect (line 4313) | func (x *AnalyzePlanResponse_JsonToDDL) ProtoReflect() protoreflect.Me...
method Descriptor (line 4326) | func (*AnalyzePlanResponse_JsonToDDL) Descriptor() ([]byte, []int) {
method GetDdlString (line 4330) | func (x *AnalyzePlanResponse_JsonToDDL) GetDdlString() string {
type ExecutePlanRequest_RequestOption (line 4337) | type ExecutePlanRequest_RequestOption struct
method Reset (line 4349) | func (x *ExecutePlanRequest_RequestOption) Reset() {
method String (line 4358) | func (x *ExecutePlanRequest_RequestOption) String() string {
method ProtoMessage (line 4362) | func (*ExecutePlanRequest_RequestOption) ProtoMessage() {}
method ProtoReflect (line 4364) | func (x *ExecutePlanRequest_RequestOption) ProtoReflect() protoreflect...
method Descriptor (line 4377) | func (*ExecutePlanRequest_RequestOption) Descriptor() ([]byte, []int) {
method GetRequestOption (line 4381) | func (m *ExecutePlanRequest_RequestOption) GetRequestOption() isExecut...
method GetReattachOptions (line 4388) | func (x *ExecutePlanRequest_RequestOption) GetReattachOptions() *Reatt...
method GetExtension (line 4395) | func (x *ExecutePlanRequest_RequestOption) GetExtension() *anypb.Any {
type isExecutePlanRequest_RequestOption_RequestOption (line 4402) | type isExecutePlanRequest_RequestOption_RequestOption interface
type ExecutePlanRequest_RequestOption_ReattachOptions (line 4406) | type ExecutePlanRequest_RequestOption_ReattachOptions struct
method isExecutePlanRequest_RequestOption_RequestOption (line 4415) | func (*ExecutePlanRequest_RequestOption_ReattachOptions) isExecutePlan...
type ExecutePlanRequest_RequestOption_Extension (line 4410) | type ExecutePlanRequest_RequestOption_Extension struct
method isExecutePlanRequest_RequestOption_RequestOption (line 4418) | func (*ExecutePlanRequest_RequestOption_Extension) isExecutePlanReques...
type ExecutePlanResponse_SqlCommandResult (line 4423) | type ExecutePlanResponse_SqlCommandResult struct
method Reset (line 4431) | func (x *ExecutePlanResponse_SqlCommandResult) Reset() {
method String (line 4440) | func (x *ExecutePlanResponse_SqlCommandResult) String() string {
method ProtoMessage (line 4444) | func (*ExecutePlanResponse_SqlCommandResult) ProtoMessage() {}
method ProtoReflect (line 4446) | func (x *ExecutePlanResponse_SqlCommandResult) ProtoReflect() protoref...
method Descriptor (line 4459) | func (*ExecutePlanResponse_SqlCommandResult) Descriptor() ([]byte, []i...
method GetRelation (line 4463) | func (x *ExecutePlanResponse_SqlCommandResult) GetRelation() *Relation {
type ExecutePlanResponse_ArrowBatch (line 4471) | type ExecutePlanResponse_ArrowBatch struct
method Reset (line 4484) | func (x *ExecutePlanResponse_ArrowBatch) Reset() {
method String (line 4493) | func (x *ExecutePlanResponse_ArrowBatch) String() string {
method ProtoMessage (line 4497) | func (*ExecutePlanResponse_ArrowBatch) ProtoMessage() {}
method ProtoReflect (line 4499) | func (x *ExecutePlanResponse_ArrowBatch) ProtoReflect() protoreflect.M...
method Descriptor (line 4512) | func (*ExecutePlanResponse_ArrowBatch) Descriptor() ([]byte, []int) {
method GetRowCount (line 4516) | func (x *ExecutePlanResponse_ArrowBatch) GetRowCount() int64 {
method GetData (line 4523) | func (x *ExecutePlanResponse_ArrowBatch) GetData() []byte {
method GetStartOffset (line 4530) | func (x *ExecutePlanResponse_ArrowBatch) GetStartOffset() int64 {
type ExecutePlanResponse_Metrics (line 4537) | type ExecutePlanResponse_Metrics struct
method Reset (line 4545) | func (x *ExecutePlanResponse_Metrics) Reset() {
method String (line 4554) | func (x *ExecutePlanResponse_Metrics) String() string {
method ProtoMessage (line 4558) | func (*ExecutePlanResponse_Metrics) ProtoMessage() {}
method ProtoReflect (line 4560) | func (x *ExecutePlanResponse_Metrics) ProtoReflect() protoreflect.Mess...
method Descriptor (line 4573) | func (*ExecutePlanResponse_Metrics) Descriptor() ([]byte, []int) {
method GetMetrics (line 4577) | func (x *ExecutePlanResponse_Metrics) GetMetrics() []*ExecutePlanRespo...
type ExecutePlanResponse_ObservedMetrics (line 4584) | type ExecutePlanResponse_ObservedMetrics struct
method Reset (line 4595) | func (x *ExecutePlanResponse_ObservedMetrics) Reset() {
method String (line 4604) | func (x *ExecutePlanResponse_ObservedMetrics) String() string {
method ProtoMessage (line 4608) | func (*ExecutePlanResponse_ObservedMetrics) ProtoMessage() {}
method ProtoReflect (line 4610) | func (x *ExecutePlanResponse_ObservedMetrics) ProtoReflect() protorefl...
method Descriptor (line 4623) | func (*ExecutePlanResponse_ObservedMetrics) Descriptor() ([]byte, []in...
method GetName (line 4627) | func (x *ExecutePlanResponse_ObservedMetrics) GetName() string {
method GetValues (line 4634) | func (x *ExecutePlanResponse_ObservedMetrics) GetValues() []*Expressio...
method GetKeys (line 4641) | func (x *ExecutePlanResponse_ObservedMetrics) GetKeys() []string {
method GetPlanId (line 4648) | func (x *ExecutePlanResponse_ObservedMetrics) GetPlanId() int64 {
type ExecutePlanResponse_ResultComplete (line 4655) | type ExecutePlanResponse_ResultComplete struct
method Reset (line 4661) | func (x *ExecutePlanResponse_ResultComplete) Reset() {
method String (line 4670) | func (x *ExecutePlanResponse_ResultComplete) String() string {
method ProtoMessage (line 4674) | func (*ExecutePlanResponse_ResultComplete) ProtoMessage() {}
method ProtoReflect (line 4676) | func (x *ExecutePlanResponse_ResultComplete) ProtoReflect() protorefle...
method Descriptor (line 4689) | func (*ExecutePlanResponse_ResultComplete) Descriptor() ([]byte, []int) {
type ExecutePlanResponse_ExecutionProgress (line 4694) | type ExecutePlanResponse_ExecutionProgress struct
method Reset (line 4705) | func (x *ExecutePlanResponse_ExecutionProgress) Reset() {
method String (line 4714) | func (x *ExecutePlanResponse_ExecutionProgress) String() string {
method ProtoMessage (line 4718) | func (*ExecutePlanResponse_ExecutionProgress) ProtoMessage() {}
method ProtoReflect (line 4720) | func (x *ExecutePlanResponse_ExecutionProgress) ProtoReflect() protore...
method Descriptor (line 4733) | func (*ExecutePlanResponse_ExecutionProgress) Descriptor() ([]byte, []...
method GetStages (line 4737) | func (x *ExecutePlanResponse_ExecutionProgress) GetStages() []*Execute...
method GetNumInflightTasks (line 4744) | func (x *ExecutePlanResponse_ExecutionProgress) GetNumInflightTasks() ...
type ExecutePlanResponse_Metrics_MetricObject (line 4751) | type ExecutePlanResponse_Metrics_MetricObject struct
method Reset (line 4762) | func (x *ExecutePlanResponse_Metrics_MetricObject) Reset() {
method String (line 4771) | func (x *ExecutePlanResponse_Metrics_MetricObject) String() string {
method ProtoMessage (line 4775) | func (*ExecutePlanResponse_Metrics_MetricObject) ProtoMessage() {}
method ProtoReflect (line 4777) | func (x *ExecutePlanResponse_Metrics_MetricObject) ProtoReflect() prot...
method Descriptor (line 4790) | func (*ExecutePlanResponse_Metrics_MetricObject) Descriptor() ([]byte,...
method GetName (line 4794) | func (x *ExecutePlanResponse_Metrics_MetricObject) GetName() string {
method GetPlanId (line 4801) | func (x *ExecutePlanResponse_Metrics_MetricObject) GetPlanId() int64 {
method GetParent (line 4808) | func (x *ExecutePlanResponse_Metrics_MetricObject) GetParent() int64 {
method GetExecutionMetrics (line 4815) | func (x *ExecutePlanResponse_Metrics_MetricObject) GetExecutionMetrics...
type ExecutePlanResponse_Metrics_MetricValue (line 4822) | type ExecutePlanResponse_Metrics_MetricValue struct
method Reset (line 4832) | func (x *ExecutePlanResponse_Metrics_MetricValue) Reset() {
method String (line 4841) | func (x *ExecutePlanResponse_Metrics_MetricValue) String() string {
method ProtoMessage (line 4845) | func (*ExecutePlanResponse_Metrics_MetricValue) ProtoMessage() {}
method ProtoReflect (line 4847) | func (x *ExecutePlanResponse_Metrics_MetricValue) ProtoReflect() proto...
method Descriptor (line 4860) | func (*ExecutePlanResponse_Metrics_MetricValue) Descriptor() ([]byte, ...
method GetName (line 4864) | func (x *ExecutePlanResponse_Metrics_MetricValue) GetName() string {
method GetValue (line 4871) | func (x *ExecutePlanResponse_Metrics_MetricValue) GetValue() int64 {
method GetMetricType (line 4878) | func (x *ExecutePlanResponse_Metrics_MetricValue) GetMetricType() stri...
type ExecutePlanResponse_ExecutionProgress_StageInfo (line 4885) | type ExecutePlanResponse_ExecutionProgress_StageInfo struct
method Reset (line 4897) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) Reset() {
method String (line 4906) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) String() str...
method ProtoMessage (line 4910) | func (*ExecutePlanResponse_ExecutionProgress_StageInfo) ProtoMessage() {}
method ProtoReflect (line 4912) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) ProtoReflect...
method Descriptor (line 4925) | func (*ExecutePlanResponse_ExecutionProgress_StageInfo) Descriptor() (...
method GetStageId (line 4929) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetStageId()...
method GetNumTasks (line 4936) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetNumTasks(...
method GetNumCompletedTasks (line 4943) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetNumComple...
method GetInputBytesRead (line 4950) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetInputByte...
method GetDone (line 4957) | func (x *ExecutePlanResponse_ExecutionProgress_StageInfo) GetDone() bo...
type ConfigRequest_Operation (line 4964) | type ConfigRequest_Operation struct
method Reset (line 4981) | func (x *ConfigRequest_Operation) Reset() {
method String (line 4990) | func (x *ConfigRequest_Operation) String() string {
method ProtoMessage (line 4994) | func (*ConfigRequest_Operation) ProtoMessage() {}
method ProtoReflect (line 4996) | func (x *ConfigRequest_Operation) ProtoReflect() protoreflect.Message {
method Descriptor (line 5009) | func (*ConfigRequest_Operation) Descriptor() ([]byte, []int) {
method GetOpType (line 5013) | func (m *ConfigRequest_Operation) GetOpType() isConfigRequest_Operatio...
method GetSet (line 5020) | func (x *ConfigRequest_Operation) GetSet() *ConfigRequest_Set {
method GetGet (line 5027) | func (x *ConfigRequest_Operation) GetGet() *ConfigRequest_Get {
method GetGetWithDefault (line 5034) | func (x *ConfigRequest_Operation) GetGetWithDefault() *ConfigRequest_G...
method GetGetOption (line 5041) | func (x *ConfigRequest_Operation) GetGetOption() *ConfigRequest_GetOpt...
method GetGetAll (line 5048) | func (x *ConfigRequest_Operation) GetGetAll() *ConfigRequest_GetAll {
method GetUnset (line 5055) | func (x *ConfigRequest_Operation) GetUnset() *ConfigRequest_Unset {
method GetIsModifiable (line 5062) | func (x *ConfigRequest_Operation) GetIsModifiable() *ConfigRequest_IsM...
type isConfigRequest_Operation_OpType (line 5069) | type isConfigRequest_Operation_OpType interface
type ConfigRequest_Operation_Set (line 5073) | type ConfigRequest_Operation_Set struct
method isConfigRequest_Operation_OpType (line 5101) | func (*ConfigRequest_Operation_Set) isConfigRequest_Operation_OpType() {}
type ConfigRequest_Operation_Get (line 5077) | type ConfigRequest_Operation_Get struct
method isConfigRequest_Operation_OpType (line 5103) | func (*ConfigRequest_Operation_Get) isConfigRequest_Operation_OpType() {}
type ConfigRequest_Operation_GetWithDefault (line 5081) | type ConfigRequest_Operation_GetWithDefault struct
method isConfigRequest_Operation_OpType (line 5105) | func (*ConfigRequest_Operation_GetWithDefault) isConfigRequest_Operati...
type ConfigRequest_Operation_GetOption (line 5085) | type ConfigRequest_Operation_GetOption struct
method isConfigRequest_Operation_OpType (line 5107) | func (*ConfigRequest_Operation_GetOption) isConfigRequest_Operation_Op...
type ConfigRequest_Operation_GetAll (line 5089) | type ConfigRequest_Operation_GetAll struct
method isConfigRequest_Operation_OpType (line 5109) | func (*ConfigRequest_Operation_GetAll) isConfigRequest_Operation_OpTyp...
type ConfigRequest_Operation_Unset (line 5093) | type ConfigRequest_Operation_Unset struct
method isConfigRequest_Operation_OpType (line 5111) | func (*ConfigRequest_Operation_Unset) isConfigRequest_Operation_OpType...
type ConfigRequest_Operation_IsModifiable (line 5097) | type ConfigRequest_Operation_IsModifiable struct
method isConfigRequest_Operation_OpType (line 5113) | func (*ConfigRequest_Operation_IsModifiable) isConfigRequest_Operation...
type ConfigRequest_Set (line 5115) | type ConfigRequest_Set struct
method Reset (line 5126) | func (x *ConfigRequest_Set) Reset() {
method String (line 5135) | func (x *ConfigRequest_Set) String() string {
method ProtoMessage (line 5139) | func (*ConfigRequest_Set) ProtoMessage() {}
method ProtoReflect (line 5141) | func (x *ConfigRequest_Set) ProtoReflect() protoreflect.Message {
method Descriptor (line 5154) | func (*ConfigRequest_Set) Descriptor() ([]byte, []int) {
method GetPairs (line 5158) | func (x *ConfigRequest_Set) GetPairs() []*KeyValue {
method GetSilent (line 5165) | func (x *ConfigRequest_Set) GetSilent() bool {
type ConfigRequest_Get (line 5172) | type ConfigRequest_Get struct
method Reset (line 5181) | func (x *ConfigRequest_Get) Reset() {
method String (line 5190) | func (x *ConfigRequest_Get) String() string {
method ProtoMessage (line 5194) | func (*ConfigRequest_Get) ProtoMessage() {}
method ProtoReflect (line 5196) | func (x *ConfigRequest_Get) ProtoReflect() protoreflect.Message {
method Descriptor (line 5209) | func (*ConfigRequest_Get) Descriptor() ([]byte, []int) {
method GetKeys (line 5213) | func (x *ConfigRequest_Get) GetKeys() []string {
type ConfigRequest_GetWithDefault (line 5220) | type ConfigRequest_GetWithDefault struct
method Reset (line 5229) | func (x *ConfigRequest_GetWithDefault) Reset() {
method String (line 5238) | func (x *ConfigRequest_GetWithDefault) String() string {
method ProtoMessage (line 5242) | func (*ConfigRequest_GetWithDefault) ProtoMessage() {}
method ProtoReflect (line 5244) | func (x *ConfigRequest_GetWithDefault) ProtoReflect() protoreflect.Mes...
method Descriptor (line 5257) | func (*ConfigRequest_GetWithDefault) Descriptor() ([]byte, []int) {
method GetPairs (line 5261) | func (x *ConfigRequest_GetWithDefault) GetPairs() []*KeyValue {
type ConfigRequest_GetOption (line 5268) | type ConfigRequest_GetOption struct
method Reset (line 5277) | func (x *ConfigRequest_GetOption) Reset() {
method String (line 5286) | func (x *ConfigRequest_GetOption) String() string {
method ProtoMessage (line 5290) | func (*ConfigRequest_GetOption) ProtoMessage() {}
method ProtoReflect (line 5292) | func (x *ConfigRequest_GetOption) ProtoReflect() protoreflect.Message {
method Descriptor (line 5305) | func (*ConfigRequest_GetOption) Descriptor() ([]byte, []int) {
method GetKeys (line 5309) | func (x *ConfigRequest_GetOption) GetKeys() []string {
type ConfigRequest_GetAll (line 5316) | type ConfigRequest_GetAll struct
method Reset (line 5325) | func (x *ConfigRequest_GetAll) Reset() {
method String (line 5334) | func (x *ConfigRequest_GetAll) String() string {
method ProtoMessage (line 5338) | func (*ConfigRequest_GetAll) ProtoMessage() {}
method ProtoReflect (line 5340) | func (x *ConfigRequest_GetAll) ProtoReflect() protoreflect.Message {
method Descriptor (line 5353) | func (*ConfigRequest_GetAll) Descriptor() ([]byte, []int) {
method GetPrefix (line 5357) | func (x *ConfigRequest_GetAll) GetPrefix() string {
type ConfigRequest_Unset (line 5364) | type ConfigRequest_Unset struct
method Reset (line 5373) | func (x *ConfigRequest_Unset) Reset() {
method String (line 5382) | func (x *ConfigRequest_Unset) String() string {
method ProtoMessage (line 5386) | func (*ConfigRequest_Unset) ProtoMessage() {}
method ProtoReflect (line 5388) | func (x *ConfigRequest_Unset) ProtoReflect() protoreflect.Message {
method Descriptor (line 5401) | func (*ConfigRequest_Unset) Descriptor() ([]byte, []int) {
method GetKeys (line 5405) | func (x *ConfigRequest_Unset) GetKeys() []string {
type ConfigRequest_IsModifiable (line 5412) | type ConfigRequest_IsModifiable struct
method Reset (line 5421) | func (x *ConfigRequest_IsModifiable) Reset() {
method String (line 5430) | func (x *ConfigRequest_IsModifiable) String() string {
method ProtoMessage (line 5434) | func (*ConfigRequest_IsModifiable) ProtoMessage() {}
method ProtoReflect (line 5436) | func (x *ConfigRequest_IsModifiable) ProtoReflect() protoreflect.Messa...
method Descriptor (line 5449) | func (*ConfigRequest_IsModifiable) Descriptor() ([]byte, []int) {
method GetKeys (line 5453) | func (x *ConfigRequest_IsModifiable) GetKeys() []string {
type AddArtifactsRequest_ArtifactChunk (line 5461) | type AddArtifactsRequest_ArtifactChunk struct
method Reset (line 5472) | func (x *AddArtifactsRequest_ArtifactChunk) Reset() {
method String (line 5481) | func (x *AddArtifactsRequest_ArtifactChunk) String() string {
method ProtoMessage (line 5485) | func (*AddArtifactsRequest_ArtifactChunk) ProtoMessage() {}
method ProtoReflect (line 5487) | func (x *AddArtifactsRequest_ArtifactChunk) ProtoReflect() protoreflec...
method Descriptor (line 5500) | func (*AddArtifactsRequest_ArtifactChunk) Descriptor() ([]byte, []int) {
method GetData (line 5504) | func (x *AddArtifactsRequest_ArtifactChunk) GetData() []byte {
method GetCrc (line 5511) | func (x *AddArtifactsRequest_ArtifactChunk) GetCrc() int64 {
type AddArtifactsRequest_SingleChunkArtifact (line 5520) | type AddArtifactsRequest_SingleChunkArtifact struct
method Reset (line 5536) | func (x *AddArtifactsRequest_SingleChunkArtifact) Reset() {
method String (line 5545) | func (x *AddArtifactsRequest_SingleChunkArtifact) String() string {
method ProtoMessage (line 5549) | func (*AddArtifactsRequest_SingleChunkArtifact) ProtoMessage() {}
method ProtoReflect (line 5551) | func (x *AddArtifactsRequest_SingleChunkArtifact) ProtoReflect() proto...
method Descriptor (line 5564) | func (*AddArtifactsRequest_SingleChunkArtifact) Descriptor() ([]byte, ...
method GetName (line 5568) | func (x *AddArtifactsRequest_SingleChunkArtifact) GetName() string {
method GetData (line 5575) | func (x *AddArtifactsRequest_SingleChunkArtifact) GetData() *AddArtifa...
type AddArtifactsRequest_Batch (line 5583) | type AddArtifactsRequest_Batch struct
method Reset (line 5591) | func (x *AddArtifactsRequest_Batch) Reset() {
method String (line 5600) | func (x *AddArtifactsRequest_Batch) String() string {
method ProtoMessage (line 5604) | func (*AddArtifactsRequest_Batch) ProtoMessage() {}
method ProtoReflect (line 5606) | func (x *AddArtifactsRequest_Batch) ProtoReflect() protoreflect.Message {
method Descriptor (line 5619) | func (*AddArtifactsRequest_Batch) Descriptor() ([]byte, []int) {
method GetArtifacts (line 5623) | func (x *AddArtifactsRequest_Batch) GetArtifacts() []*AddArtifactsRequ...
type AddArtifactsRequest_BeginChunkedArtifact (line 5633) | type AddArtifactsRequest_BeginChunkedArtifact struct
method Reset (line 5650) | func (x *AddArtifactsRequest_BeginChunkedArtifact) Reset() {
method String (line 5659) | func (x *AddArtifactsRequest_BeginChunkedArtifact) String() string {
method ProtoMessage (line 5663) | func (*AddArtifactsRequest_BeginChunkedArtifact) ProtoMessage() {}
method ProtoReflect (line 5665) | func (x *AddArtifactsRequest_BeginChunkedArtifact) ProtoReflect() prot...
method Descriptor (line 5678) | func (*AddArtifactsRequest_BeginChunkedArtifact) Descriptor() ([]byte,...
method GetName (line 5682) | func (x *AddArtifactsRequest_BeginChunkedArtifact) GetName() string {
method GetTotalBytes (line 5689) | func (x *AddArtifactsRequest_BeginChunkedArtifact) GetTotalBytes() int...
method GetNumChunks (line 5696) | func (x *AddArtifactsRequest_BeginChunkedArtifact) GetNumChunks() int64 {
method GetInitialChunk (line 5703) | func (x *AddArtifactsRequest_BeginChunkedArtifact) GetInitialChunk() *...
type AddArtifactsResponse_ArtifactSummary (line 5711) | type AddArtifactsResponse_ArtifactSummary struct
method Reset (line 5723) | func (x *AddArtifactsResponse_ArtifactSummary) Reset() {
method String (line 5732) | func (x *AddArtifactsResponse_ArtifactSummary) String() string {
method ProtoMessage (line 5736) | func (*AddArtifactsResponse_ArtifactSummary) ProtoMessage() {}
method ProtoReflect (line 5738) | func (x *AddArtifactsResponse_ArtifactSummary) ProtoReflect() protoref...
method Descriptor (line 5751) | func (*AddArtifactsResponse_ArtifactSummary) Descriptor() ([]byte, []i...
method GetName (line 5755) | func (x *AddArtifactsResponse_ArtifactSummary) GetName() string {
method GetIsCrcSuccessful (line 5762) | func (x *AddArtifactsResponse_ArtifactSummary) GetIsCrcSuccessful() bo...
type ArtifactStatusesResponse_ArtifactStatus (line 5769) | type ArtifactStatusesResponse_ArtifactStatus struct
method Reset (line 5778) | func (x *ArtifactStatusesResponse_ArtifactStatus) Reset() {
method String (line 5787) | func (x *ArtifactStatusesResponse_ArtifactStatus) String() string {
method ProtoMessage (line 5791) | func (*ArtifactStatusesResponse_ArtifactStatus) ProtoMessage() {}
method ProtoReflect (line 5793) | func (x *ArtifactStatusesResponse_ArtifactStatus) ProtoReflect() proto...
method Descriptor (line 5806) | func (*ArtifactStatusesResponse_ArtifactStatus) Descriptor() ([]byte, ...
method GetExists (line 5810) | func (x *ArtifactStatusesResponse_ArtifactStatus) GetExists() bool {
type ReleaseExecuteRequest_ReleaseAll (line 5819) | type ReleaseExecuteRequest_ReleaseAll struct
method Reset (line 5825) | func (x *ReleaseExecuteRequest_ReleaseAll) Reset() {
method String (line 5834) | func (x *ReleaseExecuteRequest_ReleaseAll) String() string {
method ProtoMessage (line 5838) | func (*ReleaseExecuteRequest_ReleaseAll) ProtoMessage() {}
method ProtoReflect (line 5840) | func (x *ReleaseExecuteRequest_ReleaseAll) ProtoReflect() protoreflect...
method Descriptor (line 5853) | func (*ReleaseExecuteRequest_ReleaseAll) Descriptor() ([]byte, []int) {
type ReleaseExecuteRequest_ReleaseUntil (line 5862) | type ReleaseExecuteRequest_ReleaseUntil struct
method Reset (line 5870) | func (x *ReleaseExecuteRequest_ReleaseUntil) Reset() {
method String (line 5879) | func (x *ReleaseExecuteRequest_ReleaseUntil) String() string {
method ProtoMessage (line 5883) | func (*ReleaseExecuteRequest_ReleaseUntil) ProtoMessage() {}
method ProtoReflect (line 5885) | func (x *ReleaseExecuteRequest_ReleaseUntil) ProtoReflect() protorefle...
method Descriptor (line 5898) | func (*ReleaseExecuteRequest_ReleaseUntil) Descriptor() ([]byte, []int) {
method GetResponseId (line 5902) | func (x *ReleaseExecuteRequest_ReleaseUntil) GetResponseId() string {
type FetchErrorDetailsResponse_StackTraceElement (line 5909) | type FetchErrorDetailsResponse_StackTraceElement struct
method Reset (line 5924) | func (x *FetchErrorDetailsResponse_StackTraceElement) Reset() {
method String (line 5933) | func (x *FetchErrorDetailsResponse_StackTraceElement) String() string {
method ProtoMessage (line 5937) | func (*FetchErrorDetailsResponse_StackTraceElement) ProtoMessage() {}
method ProtoReflect (line 5939) | func (x *FetchErrorDetailsResponse_StackTraceElement) ProtoReflect() p...
method Descriptor (line 5952) | func (*FetchErrorDetailsResponse_StackTraceElement) Descriptor() ([]by...
method GetDeclaringClass (line 5956) | func (x *FetchErrorDetailsResponse_StackTraceElement) GetDeclaringClas...
method GetMethodName (line 5963) | func (x *FetchErrorDetailsResponse_StackTraceElement) GetMethodName() ...
method GetFileName (line 5970) | func (x *FetchErrorDetailsResponse_StackTraceElement) GetFileName() st...
method GetLineNumber (line 5977) | func (x *FetchErrorDetailsResponse_StackTraceElement) GetLineNumber() ...
type FetchErrorDetailsResponse_QueryContext (line 5986) | type FetchErrorDetailsResponse_QueryContext struct
method Reset (line 6012) | func (x *FetchErrorDetailsResponse_QueryContext) Reset() {
method String (line 6021) | func (x *FetchErrorDetailsResponse_QueryContext) String() string {
method ProtoMessage (line 6025) | func (*FetchErrorDetailsResponse_QueryContext) ProtoMessage() {}
method ProtoReflect (line 6027) | func (x *FetchErrorDetailsResponse_QueryContext) ProtoReflect() protor...
method Descriptor (line 6040) | func (*FetchErrorDetailsResponse_QueryContext) Descriptor() ([]byte, [...
method GetContextType (line 6044) | func (x *FetchErrorDetailsResponse_QueryContext) GetContextType() Fetc...
method GetObjectType (line 6051) | func (x *FetchErrorDetailsResponse_QueryContext) GetObjectType() string {
method GetObjectName (line 6058) | func (x *FetchErrorDetailsResponse_QueryContext) GetObjectName() string {
method GetStartIndex (line 6065) | func (x *FetchErrorDetailsResponse_QueryContext) GetStartIndex() int32 {
method GetStopIndex (line 6072) | func (x *FetchErrorDetailsResponse_QueryContext) GetStopIndex() int32 {
method GetFragment (line 6079) | func (x *FetchErrorDetailsResponse_QueryContext) GetFragment() string {
method GetCallSite (line 6086) | func (x *FetchErrorDetailsResponse_QueryContext) GetCallSite() string {
method GetSummary (line 6093) | func (x *FetchErrorDetailsResponse_QueryContext) GetSummary() string {
type FetchErrorDetailsResponse_SparkThrowable (line 6101) | type FetchErrorDetailsResponse_SparkThrowable struct
method Reset (line 6117) | func (x *FetchErrorDetailsResponse_SparkThrowable) Reset() {
method String (line 6126) | func (x *FetchErrorDetailsResponse_SparkThrowable) String() string {
method ProtoMessage (line 6130) | func (*FetchErrorDetailsResponse_SparkThrowable) ProtoMessage() {}
method ProtoReflect (line 6132) | func (x *FetchErrorDetailsResponse_SparkThrowable) ProtoReflect() prot...
method Descriptor (line 6145) | func (*FetchErrorDetailsResponse_SparkThrowable) Descriptor() ([]byte,...
method GetErrorClass (line 6149) | func (x *FetchErrorDetailsResponse_SparkThrowable) GetErrorClass() str...
method GetMessageParameters (line 6156) | func (x *FetchErrorDetailsResponse_SparkThrowable) GetMessageParameter...
method GetQueryContexts (line 6163) | func (x *FetchErrorDetailsResponse_SparkThrowable) GetQueryContexts() ...
method GetSqlState (line 6170) | func (x *FetchErrorDetailsResponse_SparkThrowable) GetSqlState() string {
type FetchErrorDetailsResponse_Error (line 6178) | type FetchErrorDetailsResponse_Error struct
method Reset (line 6196) | func (x *FetchErrorDetailsResponse_Error) Reset() {
method String (line 6205) | func (x *FetchErrorDetailsResponse_Error) String() string {
method ProtoMessage (line 6209) | func (*FetchErrorDetailsResponse_Error) ProtoMessage() {}
method ProtoReflect (line 6211) | func (x *FetchErrorDetailsResponse_Error) ProtoReflect() protoreflect....
method Descriptor (line 6224) | func (*FetchErrorDetailsResponse_Error) Descriptor() ([]byte, []int) {
method GetErrorTypeHierarchy (line 6228) | func (x *FetchErrorDetailsResponse_Error) GetErrorTypeHierarchy() []st...
method GetMessage (line 6235) | func (x *FetchErrorDetailsResponse_Error) GetMessage() string {
method GetStackTrace (line 6242) | func (x *FetchErrorDetailsResponse_Error) GetStackTrace() []*FetchErro...
method GetCauseIdx (line 6249) | func (x *FetchErrorDetailsResponse_Error) GetCauseIdx() int32 {
method GetSparkThrowable (line 6256) | func (x *FetchErrorDetailsResponse_Error) GetSparkThrowable() *FetchEr...
function file_spark_connect_base_proto_rawDescGZIP (line 7376) | func file_spark_connect_base_proto_rawDescGZIP() []byte {
function init (line 7635) | func init() { file_spark_connect_base_proto_init() }
function file_spark_connect_base_proto_init (line 7636) | func file_spark_connect_base_proto_init() {
FILE: internal/generated/base_grpc.pb.go
constant _ (line 35) | _ = grpc.SupportPackageIsVersion7
constant SparkConnectService_ExecutePlan_FullMethodName (line 38) | SparkConnectService_ExecutePlan_FullMethodName = "/spark.connect.S...
constant SparkConnectService_AnalyzePlan_FullMethodName (line 39) | SparkConnectService_AnalyzePlan_FullMethodName = "/spark.connect.S...
constant SparkConnectService_Config_FullMethodName (line 40) | SparkConnectService_Config_FullMethodName = "/spark.connect.S...
constant SparkConnectService_AddArtifacts_FullMethodName (line 41) | SparkConnectService_AddArtifacts_FullMethodName = "/spark.connect.S...
constant SparkConnectService_ArtifactStatus_FullMethodName (line 42) | SparkConnectService_ArtifactStatus_FullMethodName = "/spark.connect.S...
constant SparkConnectService_Interrupt_FullMethodName (line 43) | SparkConnectService_Interrupt_FullMethodName = "/spark.connect.S...
constant SparkConnectService_ReattachExecute_FullMethodName (line 44) | SparkConnectService_ReattachExecute_FullMethodName = "/spark.connect.S...
constant SparkConnectService_ReleaseExecute_FullMethodName (line 45) | SparkConnectService_ReleaseExecute_FullMethodName = "/spark.connect.S...
constant SparkConnectService_ReleaseSession_FullMethodName (line 46) | SparkConnectService_ReleaseSession_FullMethodName = "/spark.connect.S...
constant SparkConnectService_FetchErrorDetails_FullMethodName (line 47) | SparkConnectService_FetchErrorDetails_FullMethodName = "/spark.connect.S...
type SparkConnectServiceClient (line 53) | type SparkConnectServiceClient interface
type sparkConnectServiceClient (line 88) | type sparkConnectServiceClient struct
method ExecutePlan (line 96) | func (c *sparkConnectServiceClient) ExecutePlan(ctx context.Context, i...
method AnalyzePlan (line 128) | func (c *sparkConnectServiceClient) AnalyzePlan(ctx context.Context, i...
method Config (line 137) | func (c *sparkConnectServiceClient) Config(ctx context.Context, in *Co...
method AddArtifacts (line 146) | func (c *sparkConnectServiceClient) AddArtifacts(ctx context.Context, ...
method ArtifactStatus (line 180) | func (c *sparkConnectServiceClient) ArtifactStatus(ctx context.Context...
method Interrupt (line 189) | func (c *sparkConnectServiceClient) Interrupt(ctx context.Context, in ...
method ReattachExecute (line 198) | func (c *sparkConnectServiceClient) ReattachExecute(ctx context.Contex...
method ReleaseExecute (line 230) | func (c *sparkConnectServiceClient) ReleaseExecute(ctx context.Context...
method ReleaseSession (line 239) | func (c *sparkConnectServiceClient) ReleaseSession(ctx context.Context...
method FetchErrorDetails (line 248) | func (c *sparkConnectServiceClient) FetchErrorDetails(ctx context.Cont...
function NewSparkConnectServiceClient (line 92) | func NewSparkConnectServiceClient(cc grpc.ClientConnInterface) SparkConn...
type SparkConnectService_ExecutePlanClient (line 111) | type SparkConnectService_ExecutePlanClient interface
type sparkConnectServiceExecutePlanClient (line 116) | type sparkConnectServiceExecutePlanClient struct
method Recv (line 120) | func (x *sparkConnectServiceExecutePlanClient) Recv() (*ExecutePlanRes...
type SparkConnectService_AddArtifactsClient (line 155) | type SparkConnectService_AddArtifactsClient interface
type sparkConnectServiceAddArtifactsClient (line 161) | type sparkConnectServiceAddArtifactsClient struct
method Send (line 165) | func (x *sparkConnectServiceAddArtifactsClient) Send(m *AddArtifactsRe...
method CloseAndRecv (line 169) | func (x *sparkConnectServiceAddArtifactsClient) CloseAndRecv() (*AddAr...
type SparkConnectService_ReattachExecuteClient (line 213) | type SparkConnectService_ReattachExecuteClient interface
type sparkConnectServiceReattachExecuteClient (line 218) | type sparkConnectServiceReattachExecuteClient struct
method Recv (line 222) | func (x *sparkConnectServiceReattachExecuteClient) Recv() (*ExecutePla...
type SparkConnectServiceServer (line 260) | type SparkConnectServiceServer interface
type UnimplementedSparkConnectServiceServer (line 297) | type UnimplementedSparkConnectServiceServer struct
method ExecutePlan (line 300) | func (UnimplementedSparkConnectServiceServer) ExecutePlan(*ExecutePlan...
method AnalyzePlan (line 303) | func (UnimplementedSparkConnectServiceServer) AnalyzePlan(context.Cont...
method Config (line 306) | func (UnimplementedSparkConnectServiceServer) Config(context.Context, ...
method AddArtifacts (line 309) | func (UnimplementedSparkConnectServiceServer) AddArtifacts(SparkConnec...
method ArtifactStatus (line 312) | func (UnimplementedSparkConnectServiceServer) ArtifactStatus(context.C...
method Interrupt (line 315) | func (UnimplementedSparkConnectServiceServer) Interrupt(context.Contex...
method ReattachExecute (line 318) | func (UnimplementedSparkConnectServiceServer) ReattachExecute(*Reattac...
method ReleaseExecute (line 321) | func (UnimplementedSparkConnectServiceServer) ReleaseExecute(context.C...
method ReleaseSession (line 324) | func (UnimplementedSparkConnectServiceServer) ReleaseSession(context.C...
method FetchErrorDetails (line 327) | func (UnimplementedSparkConnectServiceServer) FetchErrorDetails(contex...
method mustEmbedUnimplementedSparkConnectServiceServer (line 330) | func (UnimplementedSparkConnectServiceServer) mustEmbedUnimplementedSp...
type UnsafeSparkConnectServiceServer (line 335) | type UnsafeSparkConnectServiceServer interface
function RegisterSparkConnectServiceServer (line 339) | func RegisterSparkConnectServiceServer(s grpc.ServiceRegistrar, srv Spar...
function _SparkConnectService_ExecutePlan_Handler (line 343) | func _SparkConnectService_ExecutePlan_Handler(srv interface{}, stream gr...
type SparkConnectService_ExecutePlanServer (line 351) | type SparkConnectService_ExecutePlanServer interface
type sparkConnectServiceExecutePlanServer (line 356) | type sparkConnectServiceExecutePlanServer struct
method Send (line 360) | func (x *sparkConnectServiceExecutePlanServer) Send(m *ExecutePlanResp...
function _SparkConnectService_AnalyzePlan_Handler (line 364) | func _SparkConnectService_AnalyzePlan_Handler(srv interface{}, ctx conte...
function _SparkConnectService_Config_Handler (line 382) | func _SparkConnectService_Config_Handler(srv interface{}, ctx context.Co...
function _SparkConnectService_AddArtifacts_Handler (line 400) | func _SparkConnectService_AddArtifacts_Handler(srv interface{}, stream g...
type SparkConnectService_AddArtifactsServer (line 404) | type SparkConnectService_AddArtifactsServer interface
type sparkConnectServiceAddArtifactsServer (line 410) | type sparkConnectServiceAddArtifactsServer struct
method SendAndClose (line 414) | func (x *sparkConnectServiceAddArtifactsServer) SendAndClose(m *AddArt...
method Recv (line 418) | func (x *sparkConnectServiceAddArtifactsServer) Recv() (*AddArtifactsR...
function _SparkConnectService_ArtifactStatus_Handler (line 426) | func _SparkConnectService_ArtifactStatus_Handler(srv interface{}, ctx co...
function _SparkConnectService_Interrupt_Handler (line 444) | func _SparkConnectService_Interrupt_Handler(srv interface{}, ctx context...
function _SparkConnectService_ReattachExecute_Handler (line 462) | func _SparkConnectService_ReattachExecute_Handler(srv interface{}, strea...
type SparkConnectService_ReattachExecuteServer (line 470) | type SparkConnectService_ReattachExecuteServer interface
type sparkConnectServiceReattachExecuteServer (line 475) | type sparkConnectServiceReattachExecuteServer struct
method Send (line 479) | func (x *sparkConnectServiceReattachExecuteServer) Send(m *ExecutePlan...
function _SparkConnectService_ReleaseExecute_Handler (line 483) | func _SparkConnectService_ReleaseExecute_Handler(srv interface{}, ctx co...
function _SparkConnectService_ReleaseSession_Handler (line 501) | func _SparkConnectService_ReleaseSession_Handler(srv interface{}, ctx co...
function _SparkConnectService_FetchErrorDetails_Handler (line 519) | func _SparkConnectService_FetchErrorDetails_Handler(srv interface{}, ctx...
FILE: internal/generated/catalog.pb.go
constant _ (line 34) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 36) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type Catalog (line 40) | type Catalog struct
method Reset (line 76) | func (x *Catalog) Reset() {
method String (line 85) | func (x *Catalog) String() string {
method ProtoMessage (line 89) | func (*Catalog) ProtoMessage() {}
method ProtoReflect (line 91) | func (x *Catalog) ProtoReflect() protoreflect.Message {
method Descriptor (line 104) | func (*Catalog) Descriptor() ([]byte, []int) {
method GetCatType (line 108) | func (m *Catalog) GetCatType() isCatalog_CatType {
method GetCurrentDatabase (line 115) | func (x *Catalog) GetCurrentDatabase() *CurrentDatabase {
method GetSetCurrentDatabase (line 122) | func (x *Catalog) GetSetCurrentDatabase() *SetCurrentDatabase {
method GetListDatabases (line 129) | func (x *Catalog) GetListDatabases() *ListDatabases {
method GetListTables (line 136) | func (x *Catalog) GetListTables() *ListTables {
method GetListFunctions (line 143) | func (x *Catalog) GetListFunctions() *ListFunctions {
method GetListColumns (line 150) | func (x *Catalog) GetListColumns() *ListColumns {
method GetGetDatabase (line 157) | func (x *Catalog) GetGetDatabase() *GetDatabase {
method GetGetTable (line 164) | func (x *Catalog) GetGetTable() *GetTable {
method GetGetFunction (line 171) | func (x *Catalog) GetGetFunction() *GetFunction {
method GetDatabaseExists (line 178) | func (x *Catalog) GetDatabaseExists() *DatabaseExists {
method GetTableExists (line 185) | func (x *Catalog) GetTableExists() *TableExists {
method GetFunctionExists (line 192) | func (x *Catalog) GetFunctionExists() *FunctionExists {
method GetCreateExternalTable (line 199) | func (x *Catalog) GetCreateExternalTable() *CreateExternalTable {
method GetCreateTable (line 206) | func (x *Catalog) GetCreateTable() *CreateTable {
method GetDropTempView (line 213) | func (x *Catalog) GetDropTempView() *DropTempView {
method GetDropGlobalTempView (line 220) | func (x *Catalog) GetDropGlobalTempView() *DropGlobalTempView {
method GetRecoverPartitions (line 227) | func (x *Catalog) GetRecoverPartitions() *RecoverPartitions {
method GetIsCached (line 234) | func (x *Catalog) GetIsCached() *IsCached {
method GetCacheTable (line 241) | func (x *Catalog) GetCacheTable() *CacheTable {
method GetUncacheTable (line 248) | func (x *Catalog) GetUncacheTable() *UncacheTable {
method GetClearCache (line 255) | func (x *Catalog) GetClearCache() *ClearCache {
method GetRefreshTable (line 262) | func (x *Catalog) GetRefreshTable() *RefreshTable {
method GetRefreshByPath (line 269) | func (x *Catalog) GetRefreshByPath() *RefreshByPath {
method GetCurrentCatalog (line 276) | func (x *Catalog) GetCurrentCatalog() *CurrentCatalog {
method GetSetCurrentCatalog (line 283) | func (x *Catalog) GetSetCurrentCatalog() *SetCurrentCatalog {
method GetListCatalogs (line 290) | func (x *Catalog) GetListCatalogs() *ListCatalogs {
type isCatalog_CatType (line 297) | type isCatalog_CatType interface
type Catalog_CurrentDatabase (line 301) | type Catalog_CurrentDatabase struct
method isCatalog_CatType (line 405) | func (*Catalog_CurrentDatabase) isCatalog_CatType() {}
type Catalog_SetCurrentDatabase (line 305) | type Catalog_SetCurrentDatabase struct
method isCatalog_CatType (line 407) | func (*Catalog_SetCurrentDatabase) isCatalog_CatType() {}
type Catalog_ListDatabases (line 309) | type Catalog_ListDatabases struct
method isCatalog_CatType (line 409) | func (*Catalog_ListDatabases) isCatalog_CatType() {}
type Catalog_ListTables (line 313) | type Catalog_ListTables struct
method isCatalog_CatType (line 411) | func (*Catalog_ListTables) isCatalog_CatType() {}
type Catalog_ListFunctions (line 317) | type Catalog_ListFunctions struct
method isCatalog_CatType (line 413) | func (*Catalog_ListFunctions) isCatalog_CatType() {}
type Catalog_ListColumns (line 321) | type Catalog_ListColumns struct
method isCatalog_CatType (line 415) | func (*Catalog_ListColumns) isCatalog_CatType() {}
type Catalog_GetDatabase (line 325) | type Catalog_GetDatabase struct
method isCatalog_CatType (line 417) | func (*Catalog_GetDatabase) isCatalog_CatType() {}
type Catalog_GetTable (line 329) | type Catalog_GetTable struct
method isCatalog_CatType (line 419) | func (*Catalog_GetTable) isCatalog_CatType() {}
type Catalog_GetFunction (line 333) | type Catalog_GetFunction struct
method isCatalog_CatType (line 421) | func (*Catalog_GetFunction) isCatalog_CatType() {}
type Catalog_DatabaseExists (line 337) | type Catalog_DatabaseExists struct
method isCatalog_CatType (line 423) | func (*Catalog_DatabaseExists) isCatalog_CatType() {}
type Catalog_TableExists (line 341) | type Catalog_TableExists struct
method isCatalog_CatType (line 425) | func (*Catalog_TableExists) isCatalog_CatType() {}
type Catalog_FunctionExists (line 345) | type Catalog_FunctionExists struct
method isCatalog_CatType (line 427) | func (*Catalog_FunctionExists) isCatalog_CatType() {}
type Catalog_CreateExternalTable (line 349) | type Catalog_CreateExternalTable struct
method isCatalog_CatType (line 429) | func (*Catalog_CreateExternalTable) isCatalog_CatType() {}
type Catalog_CreateTable (line 353) | type Catalog_CreateTable struct
method isCatalog_CatType (line 431) | func (*Catalog_CreateTable) isCatalog_CatType() {}
type Catalog_DropTempView (line 357) | type Catalog_DropTempView struct
method isCatalog_CatType (line 433) | func (*Catalog_DropTempView) isCatalog_CatType() {}
type Catalog_DropGlobalTempView (line 361) | type Catalog_DropGlobalTempView struct
method isCatalog_CatType (line 435) | func (*Catalog_DropGlobalTempView) isCatalog_CatType() {}
type Catalog_RecoverPartitions (line 365) | type Catalog_RecoverPartitions struct
method isCatalog_CatType (line 437) | func (*Catalog_RecoverPartitions) isCatalog_CatType() {}
type Catalog_IsCached (line 369) | type Catalog_IsCached struct
method isCatalog_CatType (line 439) | func (*Catalog_IsCached) isCatalog_CatType() {}
type Catalog_CacheTable (line 373) | type Catalog_CacheTable struct
method isCatalog_CatType (line 441) | func (*Catalog_CacheTable) isCatalog_CatType() {}
type Catalog_UncacheTable (line 377) | type Catalog_UncacheTable struct
method isCatalog_CatType (line 443) | func (*Catalog_UncacheTable) isCatalog_CatType() {}
type Catalog_ClearCache (line 381) | type Catalog_ClearCache struct
method isCatalog_CatType (line 445) | func (*Catalog_ClearCache) isCatalog_CatType() {}
type Catalog_RefreshTable (line 385) | type Catalog_RefreshTable struct
method isCatalog_CatType (line 447) | func (*Catalog_RefreshTable) isCatalog_CatType() {}
type Catalog_RefreshByPath (line 389) | type Catalog_RefreshByPath struct
method isCatalog_CatType (line 449) | func (*Catalog_RefreshByPath) isCatalog_CatType() {}
type Catalog_CurrentCatalog (line 393) | type Catalog_CurrentCatalog struct
method isCatalog_CatType (line 451) | func (*Catalog_CurrentCatalog) isCatalog_CatType() {}
type Catalog_SetCurrentCatalog (line 397) | type Catalog_SetCurrentCatalog struct
method isCatalog_CatType (line 453) | func (*Catalog_SetCurrentCatalog) isCatalog_CatType() {}
type Catalog_ListCatalogs (line 401) | type Catalog_ListCatalogs struct
method isCatalog_CatType (line 455) | func (*Catalog_ListCatalogs) isCatalog_CatType() {}
type CurrentDatabase (line 458) | type CurrentDatabase struct
method Reset (line 464) | func (x *CurrentDatabase) Reset() {
method String (line 473) | func (x *CurrentDatabase) String() string {
method ProtoMessage (line 477) | func (*CurrentDatabase) ProtoMessage() {}
method ProtoReflect (line 479) | func (x *CurrentDatabase) ProtoReflect() protoreflect.Message {
method Descriptor (line 492) | func (*CurrentDatabase) Descriptor() ([]byte, []int) {
type SetCurrentDatabase (line 497) | type SetCurrentDatabase struct
method Reset (line 506) | func (x *SetCurrentDatabase) Reset() {
method String (line 515) | func (x *SetCurrentDatabase) String() string {
method ProtoMessage (line 519) | func (*SetCurrentDatabase) ProtoMessage() {}
method ProtoReflect (line 521) | func (x *SetCurrentDatabase) ProtoReflect() protoreflect.Message {
method Descriptor (line 534) | func (*SetCurrentDatabase) Descriptor() ([]byte, []int) {
method GetDbName (line 538) | func (x *SetCurrentDatabase) GetDbName() string {
type ListDatabases (line 546) | type ListDatabases struct
method Reset (line 555) | func (x *ListDatabases) Reset() {
method String (line 564) | func (x *ListDatabases) String() string {
method ProtoMessage (line 568) | func (*ListDatabases) ProtoMessage() {}
method ProtoReflect (line 570) | func (x *ListDatabases) ProtoReflect() protoreflect.Message {
method Descriptor (line 583) | func (*ListDatabases) Descriptor() ([]byte, []int) {
method GetPattern (line 587) | func (x *ListDatabases) GetPattern() string {
type ListTables (line 595) | type ListTables struct
method Reset (line 606) | func (x *ListTables) Reset() {
method String (line 615) | func (x *ListTables) String() string {
method ProtoMessage (line 619) | func (*ListTables) ProtoMessage() {}
method ProtoReflect (line 621) | func (x *ListTables) ProtoReflect() protoreflect.Message {
method Descriptor (line 634) | func (*ListTables) Descriptor() ([]byte, []int) {
method GetDbName (line 638) | func (x *ListTables) GetDbName() string {
method GetPattern (line 645) | func (x *ListTables) GetPattern() string {
type ListFunctions (line 653) | type ListFunctions struct
method Reset (line 664) | func (x *ListFunctions) Reset() {
method String (line 673) | func (x *ListFunctions) String() string {
method ProtoMessage (line 677) | func (*ListFunctions) ProtoMessage() {}
method ProtoReflect (line 679) | func (x *ListFunctions) ProtoReflect() protoreflect.Message {
method Descriptor (line 692) | func (*ListFunctions) Descriptor() ([]byte, []int) {
method GetDbName (line 696) | func (x *ListFunctions) GetDbName() string {
method GetPattern (line 703) | func (x *ListFunctions) GetPattern() string {
type ListColumns (line 711) | type ListColumns struct
method Reset (line 722) | func (x *ListColumns) Reset() {
method String (line 731) | func (x *ListColumns) String() string {
method ProtoMessage (line 735) | func (*ListColumns) ProtoMessage() {}
method ProtoReflect (line 737) | func (x *ListColumns) ProtoReflect() protoreflect.Message {
method Descriptor (line 750) | func (*ListColumns) Descriptor() ([]byte, []int) {
method GetTableName (line 754) | func (x *ListColumns) GetTableName() string {
method GetDbName (line 761) | func (x *ListColumns) GetDbName() string {
type GetDatabase (line 769) | type GetDatabase struct
method Reset (line 778) | func (x *GetDatabase) Reset() {
method String (line 787) | func (x *GetDatabase) String() string {
method ProtoMessage (line 791) | func (*GetDatabase) ProtoMessage() {}
method ProtoReflect (line 793) | func (x *GetDatabase) ProtoReflect() protoreflect.Message {
method Descriptor (line 806) | func (*GetDatabase) Descriptor() ([]byte, []int) {
method GetDbName (line 810) | func (x *GetDatabase) GetDbName() string {
type GetTable (line 818) | type GetTable struct
method Reset (line 829) | func (x *GetTable) Reset() {
method String (line 838) | func (x *GetTable) String() string {
method ProtoMessage (line 842) | func (*GetTable) ProtoMessage() {}
method ProtoReflect (line 844) | func (x *GetTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 857) | func (*GetTable) Descriptor() ([]byte, []int) {
method GetTableName (line 861) | func (x *GetTable) GetTableName() string {
method GetDbName (line 868) | func (x *GetTable) GetDbName() string {
type GetFunction (line 876) | type GetFunction struct
method Reset (line 887) | func (x *GetFunction) Reset() {
method String (line 896) | func (x *GetFunction) String() string {
method ProtoMessage (line 900) | func (*GetFunction) ProtoMessage() {}
method ProtoReflect (line 902) | func (x *GetFunction) ProtoReflect() protoreflect.Message {
method Descriptor (line 915) | func (*GetFunction) Descriptor() ([]byte, []int) {
method GetFunctionName (line 919) | func (x *GetFunction) GetFunctionName() string {
method GetDbName (line 926) | func (x *GetFunction) GetDbName() string {
type DatabaseExists (line 934) | type DatabaseExists struct
method Reset (line 943) | func (x *DatabaseExists) Reset() {
method String (line 952) | func (x *DatabaseExists) String() string {
method ProtoMessage (line 956) | func (*DatabaseExists) ProtoMessage() {}
method ProtoReflect (line 958) | func (x *DatabaseExists) ProtoReflect() protoreflect.Message {
method Descriptor (line 971) | func (*DatabaseExists) Descriptor() ([]byte, []int) {
method GetDbName (line 975) | func (x *DatabaseExists) GetDbName() string {
type TableExists (line 983) | type TableExists struct
method Reset (line 994) | func (x *TableExists) Reset() {
method String (line 1003) | func (x *TableExists) String() string {
method ProtoMessage (line 1007) | func (*TableExists) ProtoMessage() {}
method ProtoReflect (line 1009) | func (x *TableExists) ProtoReflect() protoreflect.Message {
method Descriptor (line 1022) | func (*TableExists) Descriptor() ([]byte, []int) {
method GetTableName (line 1026) | func (x *TableExists) GetTableName() string {
method GetDbName (line 1033) | func (x *TableExists) GetDbName() string {
type FunctionExists (line 1041) | type FunctionExists struct
method Reset (line 1052) | func (x *FunctionExists) Reset() {
method String (line 1061) | func (x *FunctionExists) String() string {
method ProtoMessage (line 1065) | func (*FunctionExists) ProtoMessage() {}
method ProtoReflect (line 1067) | func (x *FunctionExists) ProtoReflect() protoreflect.Message {
method Descriptor (line 1080) | func (*FunctionExists) Descriptor() ([]byte, []int) {
method GetFunctionName (line 1084) | func (x *FunctionExists) GetFunctionName() string {
method GetDbName (line 1091) | func (x *FunctionExists) GetDbName() string {
type CreateExternalTable (line 1099) | type CreateExternalTable struct
method Reset (line 1117) | func (x *CreateExternalTable) Reset() {
method String (line 1126) | func (x *CreateExternalTable) String() string {
method ProtoMessage (line 1130) | func (*CreateExternalTable) ProtoMessage() {}
method ProtoReflect (line 1132) | func (x *CreateExternalTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 1145) | func (*CreateExternalTable) Descriptor() ([]byte, []int) {
method GetTableName (line 1149) | func (x *CreateExternalTable) GetTableName() string {
method GetPath (line 1156) | func (x *CreateExternalTable) GetPath() string {
method GetSource (line 1163) | func (x *CreateExternalTable) GetSource() string {
method GetSchema (line 1170) | func (x *CreateExternalTable) GetSchema() *DataType {
method GetOptions (line 1177) | func (x *CreateExternalTable) GetOptions() map[string]string {
type CreateTable (line 1185) | type CreateTable struct
method Reset (line 1205) | func (x *CreateTable) Reset() {
method String (line 1214) | func (x *CreateTable) String() string {
method ProtoMessage (line 1218) | func (*CreateTable) ProtoMessage() {}
method ProtoReflect (line 1220) | func (x *CreateTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 1233) | func (*CreateTable) Descriptor() ([]byte, []int) {
method GetTableName (line 1237) | func (x *CreateTable) GetTableName() string {
method GetPath (line 1244) | func (x *CreateTable) GetPath() string {
method GetSource (line 1251) | func (x *CreateTable) GetSource() string {
method GetDescription (line 1258) | func (x *CreateTable) GetDescription() string {
method GetSchema (line 1265) | func (x *CreateTable) GetSchema() *DataType {
method GetOptions (line 1272) | func (x *CreateTable) GetOptions() map[string]string {
type DropTempView (line 1280) | type DropTempView struct
method Reset (line 1289) | func (x *DropTempView) Reset() {
method String (line 1298) | func (x *DropTempView) String() string {
method ProtoMessage (line 1302) | func (*DropTempView) ProtoMessage() {}
method ProtoReflect (line 1304) | func (x *DropTempView) ProtoReflect() protoreflect.Message {
method Descriptor (line 1317) | func (*DropTempView) Descriptor() ([]byte, []int) {
method GetViewName (line 1321) | func (x *DropTempView) GetViewName() string {
type DropGlobalTempView (line 1329) | type DropGlobalTempView struct
method Reset (line 1338) | func (x *DropGlobalTempView) Reset() {
method String (line 1347) | func (x *DropGlobalTempView) String() string {
method ProtoMessage (line 1351) | func (*DropGlobalTempView) ProtoMessage() {}
method ProtoReflect (line 1353) | func (x *DropGlobalTempView) ProtoReflect() protoreflect.Message {
method Descriptor (line 1366) | func (*DropGlobalTempView) Descriptor() ([]byte, []int) {
method GetViewName (line 1370) | func (x *DropGlobalTempView) GetViewName() string {
type RecoverPartitions (line 1378) | type RecoverPartitions struct
method Reset (line 1387) | func (x *RecoverPartitions) Reset() {
method String (line 1396) | func (x *RecoverPartitions) String() string {
method ProtoMessage (line 1400) | func (*RecoverPartitions) ProtoMessage() {}
method ProtoReflect (line 1402) | func (x *RecoverPartitions) ProtoReflect() protoreflect.Message {
method Descriptor (line 1415) | func (*RecoverPartitions) Descriptor() ([]byte, []int) {
method GetTableName (line 1419) | func (x *RecoverPartitions) GetTableName() string {
type IsCached (line 1427) | type IsCached struct
method Reset (line 1436) | func (x *IsCached) Reset() {
method String (line 1445) | func (x *IsCached) String() string {
method ProtoMessage (line 1449) | func (*IsCached) ProtoMessage() {}
method ProtoReflect (line 1451) | func (x *IsCached) ProtoReflect() protoreflect.Message {
method Descriptor (line 1464) | func (*IsCached) Descriptor() ([]byte, []int) {
method GetTableName (line 1468) | func (x *IsCached) GetTableName() string {
type CacheTable (line 1476) | type CacheTable struct
method Reset (line 1487) | func (x *CacheTable) Reset() {
method String (line 1496) | func (x *CacheTable) String() string {
method ProtoMessage (line 1500) | func (*CacheTable) ProtoMessage() {}
method ProtoReflect (line 1502) | func (x *CacheTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 1515) | func (*CacheTable) Descriptor() ([]byte, []int) {
method GetTableName (line 1519) | func (x *CacheTable) GetTableName() string {
method GetStorageLevel (line 1526) | func (x *CacheTable) GetStorageLevel() *StorageLevel {
type UncacheTable (line 1534) | type UncacheTable struct
method Reset (line 1543) | func (x *UncacheTable) Reset() {
method String (line 1552) | func (x *UncacheTable) String() string {
method ProtoMessage (line 1556) | func (*UncacheTable) ProtoMessage() {}
method ProtoReflect (line 1558) | func (x *UncacheTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 1571) | func (*UncacheTable) Descriptor() ([]byte, []int) {
method GetTableName (line 1575) | func (x *UncacheTable) GetTableName() string {
type ClearCache (line 1583) | type ClearCache struct
method Reset (line 1589) | func (x *ClearCache) Reset() {
method String (line 1598) | func (x *ClearCache) String() string {
method ProtoMessage (line 1602) | func (*ClearCache) ProtoMessage() {}
method ProtoReflect (line 1604) | func (x *ClearCache) ProtoReflect() protoreflect.Message {
method Descriptor (line 1617) | func (*ClearCache) Descriptor() ([]byte, []int) {
type RefreshTable (line 1622) | type RefreshTable struct
method Reset (line 1631) | func (x *RefreshTable) Reset() {
method String (line 1640) | func (x *RefreshTable) String() string {
method ProtoMessage (line 1644) | func (*RefreshTable) ProtoMessage() {}
method ProtoReflect (line 1646) | func (x *RefreshTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 1659) | func (*RefreshTable) Descriptor() ([]byte, []int) {
method GetTableName (line 1663) | func (x *RefreshTable) GetTableName() string {
type RefreshByPath (line 1671) | type RefreshByPath struct
method Reset (line 1680) | func (x *RefreshByPath) Reset() {
method String (line 1689) | func (x *RefreshByPath) String() string {
method ProtoMessage (line 1693) | func (*RefreshByPath) ProtoMessage() {}
method ProtoReflect (line 1695) | func (x *RefreshByPath) ProtoReflect() protoreflect.Message {
method Descriptor (line 1708) | func (*RefreshByPath) Descriptor() ([]byte, []int) {
method GetPath (line 1712) | func (x *RefreshByPath) GetPath() string {
type CurrentCatalog (line 1720) | type CurrentCatalog struct
method Reset (line 1726) | func (x *CurrentCatalog) Reset() {
method String (line 1735) | func (x *CurrentCatalog) String() string {
method ProtoMessage (line 1739) | func (*CurrentCatalog) ProtoMessage() {}
method ProtoReflect (line 1741) | func (x *CurrentCatalog) ProtoReflect() protoreflect.Message {
method Descriptor (line 1754) | func (*CurrentCatalog) Descriptor() ([]byte, []int) {
type SetCurrentCatalog (line 1759) | type SetCurrentCatalog struct
method Reset (line 1768) | func (x *SetCurrentCatalog) Reset() {
method String (line 1777) | func (x *SetCurrentCatalog) String() string {
method ProtoMessage (line 1781) | func (*SetCurrentCatalog) ProtoMessage() {}
method ProtoReflect (line 1783) | func (x *SetCurrentCatalog) ProtoReflect() protoreflect.Message {
method Descriptor (line 1796) | func (*SetCurrentCatalog) Descriptor() ([]byte, []int) {
method GetCatalogName (line 1800) | func (x *SetCurrentCatalog) GetCatalogName() string {
type ListCatalogs (line 1808) | type ListCatalogs struct
method Reset (line 1817) | func (x *ListCatalogs) Reset() {
method String (line 1826) | func (x *ListCatalogs) String() string {
method ProtoMessage (line 1830) | func (*ListCatalogs) ProtoMessage() {}
method ProtoReflect (line 1832) | func (x *ListCatalogs) ProtoReflect() protoreflect.Message {
method Descriptor (line 1845) | func (*ListCatalogs) Descriptor() ([]byte, []int) {
method GetPattern (line 1849) | func (x *ListCatalogs) GetPattern() string {
function file_spark_connect_catalog_proto_rawDescGZIP (line 2128) | func file_spark_connect_catalog_proto_rawDescGZIP() []byte {
function init (line 2208) | func init() { file_spark_connect_catalog_proto_init() }
function file_spark_connect_catalog_proto_init (line 2209) | func file_spark_connect_catalog_proto_init() {
FILE: internal/generated/commands.pb.go
constant _ (line 35) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 37) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type StreamingQueryEventType (line 43) | type StreamingQueryEventType
method Enum (line 68) | func (x StreamingQueryEventType) Enum() *StreamingQueryEventType {
method String (line 74) | func (x StreamingQueryEventType) String() string {
method Descriptor (line 78) | func (StreamingQueryEventType) Descriptor() protoreflect.EnumDescriptor {
method Type (line 82) | func (StreamingQueryEventType) Type() protoreflect.EnumType {
method Number (line 86) | func (x StreamingQueryEventType) Number() protoreflect.EnumNumber {
method EnumDescriptor (line 91) | func (StreamingQueryEventType) EnumDescriptor() ([]byte, []int) {
constant StreamingQueryEventType_QUERY_PROGRESS_UNSPECIFIED (line 46) | StreamingQueryEventType_QUERY_PROGRESS_UNSPECIFIED StreamingQueryEventTy...
constant StreamingQueryEventType_QUERY_PROGRESS_EVENT (line 47) | StreamingQueryEventType_QUERY_PROGRESS_EVENT StreamingQueryEventTy...
constant StreamingQueryEventType_QUERY_TERMINATED_EVENT (line 48) | StreamingQueryEventType_QUERY_TERMINATED_EVENT StreamingQueryEventTy...
constant StreamingQueryEventType_QUERY_IDLE_EVENT (line 49) | StreamingQueryEventType_QUERY_IDLE_EVENT StreamingQueryEventTy...
type WriteOperation_SaveMode (line 95) | type WriteOperation_SaveMode
method Enum (line 123) | func (x WriteOperation_SaveMode) Enum() *WriteOperation_SaveMode {
method String (line 129) | func (x WriteOperation_SaveMode) String() string {
method Descriptor (line 133) | func (WriteOperation_SaveMode) Descriptor() protoreflect.EnumDescriptor {
method Type (line 137) | func (WriteOperation_SaveMode) Type() protoreflect.EnumType {
method Number (line 141) | func (x WriteOperation_SaveMode) Number() protoreflect.EnumNumber {
method EnumDescriptor (line 146) | func (WriteOperation_SaveMode) EnumDescriptor() ([]byte, []int) {
constant WriteOperation_SAVE_MODE_UNSPECIFIED (line 98) | WriteOperation_SAVE_MODE_UNSPECIFIED WriteOperation_SaveMode = 0
constant WriteOperation_SAVE_MODE_APPEND (line 99) | WriteOperation_SAVE_MODE_APPEND WriteOperation_SaveMode = 1
constant WriteOperation_SAVE_MODE_OVERWRITE (line 100) | WriteOperation_SAVE_MODE_OVERWRITE WriteOperation_SaveMode = 2
constant WriteOperation_SAVE_MODE_ERROR_IF_EXISTS (line 101) | WriteOperation_SAVE_MODE_ERROR_IF_EXISTS WriteOperation_SaveMode = 3
constant WriteOperation_SAVE_MODE_IGNORE (line 102) | WriteOperation_SAVE_MODE_IGNORE WriteOperation_SaveMode = 4
type WriteOperation_SaveTable_TableSaveMethod (line 150) | type WriteOperation_SaveTable_TableSaveMethod
method Enum (line 172) | func (x WriteOperation_SaveTable_TableSaveMethod) Enum() *WriteOperati...
method String (line 178) | func (x WriteOperation_SaveTable_TableSaveMethod) String() string {
method Descriptor (line 182) | func (WriteOperation_SaveTable_TableSaveMethod) Descriptor() protorefl...
method Type (line 186) | func (WriteOperation_SaveTable_TableSaveMethod) Type() protoreflect.En...
method Number (line 190) | func (x WriteOperation_SaveTable_TableSaveMethod) Number() protoreflec...
method EnumDescriptor (line 195) | func (WriteOperation_SaveTable_TableSaveMethod) EnumDescriptor() ([]by...
constant WriteOperation_SaveTable_TABLE_SAVE_METHOD_UNSPECIFIED (line 153) | WriteOperation_SaveTable_TABLE_SAVE_METHOD_UNSPECIFIED WriteOperation_...
constant WriteOperation_SaveTable_TABLE_SAVE_METHOD_SAVE_AS_TABLE (line 154) | WriteOperation_SaveTable_TABLE_SAVE_METHOD_SAVE_AS_TABLE WriteOperation_...
constant WriteOperation_SaveTable_TABLE_SAVE_METHOD_INSERT_INTO (line 155) | WriteOperation_SaveTable_TABLE_SAVE_METHOD_INSERT_INTO WriteOperation_...
type WriteOperationV2_Mode (line 199) | type WriteOperationV2_Mode
method Enum (line 233) | func (x WriteOperationV2_Mode) Enum() *WriteOperationV2_Mode {
method String (line 239) | func (x WriteOperationV2_Mode) String() string {
method Descriptor (line 243) | func (WriteOperationV2_Mode) Descriptor() protoreflect.EnumDescriptor {
method Type (line 247) | func (WriteOperationV2_Mode) Type() protoreflect.EnumType {
method Number (line 251) | func (x WriteOperationV2_Mode) Number() protoreflect.EnumNumber {
method EnumDescriptor (line 256) | func (WriteOperationV2_Mode) EnumDescriptor() ([]byte, []int) {
constant WriteOperationV2_MODE_UNSPECIFIED (line 202) | WriteOperationV2_MODE_UNSPECIFIED WriteOperationV2_Mode = 0
constant WriteOperationV2_MODE_CREATE (line 203) | WriteOperationV2_MODE_CREATE WriteOperationV2_Mode = 1
constant WriteOperationV2_MODE_OVERWRITE (line 204) | WriteOperationV2_MODE_OVERWRITE WriteOperationV2_Mode = 2
constant WriteOperationV2_MODE_OVERWRITE_PARTITIONS (line 205) | WriteOperationV2_MODE_OVERWRITE_PARTITIONS WriteOperationV2_Mode = 3
constant WriteOperationV2_MODE_APPEND (line 206) | WriteOperationV2_MODE_APPEND WriteOperationV2_Mode = 4
constant WriteOperationV2_MODE_REPLACE (line 207) | WriteOperationV2_MODE_REPLACE WriteOperationV2_Mode = 5
constant WriteOperationV2_MODE_CREATE_OR_REPLACE (line 208) | WriteOperationV2_MODE_CREATE_OR_REPLACE WriteOperationV2_Mode = 6
type Command (line 262) | type Command struct
method Reset (line 292) | func (x *Command) Reset() {
method String (line 301) | func (x *Command) String() string {
method ProtoMessage (line 305) | func (*Command) ProtoMessage() {}
method ProtoReflect (line 307) | func (x *Command) ProtoReflect() protoreflect.Message {
method Descriptor (line 320) | func (*Command) Descriptor() ([]byte, []int) {
method GetCommandType (line 324) | func (m *Command) GetCommandType() isCommand_CommandType {
method GetRegisterFunction (line 331) | func (x *Command) GetRegisterFunction() *CommonInlineUserDefinedFuncti...
method GetWriteOperation (line 338) | func (x *Command) GetWriteOperation() *WriteOperation {
method GetCreateDataframeView (line 345) | func (x *Command) GetCreateDataframeView() *CreateDataFrameViewCommand {
method GetWriteOperationV2 (line 352) | func (x *Command) GetWriteOperationV2() *WriteOperationV2 {
method GetSqlCommand (line 359) | func (x *Command) GetSqlCommand() *SqlCommand {
method GetWriteStreamOperationStart (line 366) | func (x *Command) GetWriteStreamOperationStart() *WriteStreamOperation...
method GetStreamingQueryCommand (line 373) | func (x *Command) GetStreamingQueryCommand() *StreamingQueryCommand {
method GetGetResourcesCommand (line 380) | func (x *Command) GetGetResourcesCommand() *GetResourcesCommand {
method GetStreamingQueryManagerCommand (line 387) | func (x *Command) GetStreamingQueryManagerCommand() *StreamingQueryMan...
method GetRegisterTableFunction (line 394) | func (x *Command) GetRegisterTableFunction() *CommonInlineUserDefinedT...
method GetStreamingQueryListenerBusCommand (line 401) | func (x *Command) GetStreamingQueryListenerBusCommand() *StreamingQuer...
method GetRegisterDataSource (line 408) | func (x *Command) GetRegisterDataSource() *CommonInlineUserDefinedData...
method GetCreateResourceProfileCommand (line 415) | func (x *Command) GetCreateResourceProfileCommand() *CreateResourcePro...
method GetCheckpointCommand (line 422) | func (x *Command) GetCheckpointCommand() *CheckpointCommand {
method GetRemoveCachedRemoteRelationCommand (line 429) | func (x *Command) GetRemoveCachedRemoteRelationCommand() *RemoveCached...
method GetMergeIntoTableCommand (line 436) | func (x *Command) GetMergeIntoTableCommand() *MergeIntoTableCommand {
method GetMlCommand (line 443) | func (x *Command) GetMlCommand() *MlCommand {
method GetExecuteExternalCommand (line 450) | func (x *Command) GetExecuteExternalCommand() *ExecuteExternalCommand {
method GetPipelineCommand (line 457) | func (x *Command) GetPipelineCommand() *PipelineCommand {
method GetExtension (line 464) | func (x *Command) GetExtension() *anypb.Any {
type isCommand_CommandType (line 471) | type isCommand_CommandType interface
type Command_RegisterFunction (line 475) | type Command_RegisterFunction struct
method isCommand_CommandType (line 557) | func (*Command_RegisterFunction) isCommand_CommandType() {}
type Command_WriteOperation (line 479) | type Command_WriteOperation struct
method isCommand_CommandType (line 559) | func (*Command_WriteOperation) isCommand_CommandType() {}
type Command_CreateDataframeView (line 483) | type Command_CreateDataframeView struct
method isCommand_CommandType (line 561) | func (*Command_CreateDataframeView) isCommand_CommandType() {}
type Command_WriteOperationV2 (line 487) | type Command_WriteOperationV2 struct
method isCommand_CommandType (line 563) | func (*Command_WriteOperationV2) isCommand_CommandType() {}
type Command_SqlCommand (line 491) | type Command_SqlCommand struct
method isCommand_CommandType (line 565) | func (*Command_SqlCommand) isCommand_CommandType() {}
type Command_WriteStreamOperationStart (line 495) | type Command_WriteStreamOperationStart struct
method isCommand_CommandType (line 567) | func (*Command_WriteStreamOperationStart) isCommand_CommandType() {}
type Command_StreamingQueryCommand (line 499) | type Command_StreamingQueryCommand struct
method isCommand_CommandType (line 569) | func (*Command_StreamingQueryCommand) isCommand_CommandType() {}
type Command_GetResourcesCommand (line 503) | type Command_GetResourcesCommand struct
method isCommand_CommandType (line 571) | func (*Command_GetResourcesCommand) isCommand_CommandType() {}
type Command_StreamingQueryManagerCommand (line 507) | type Command_StreamingQueryManagerCommand struct
method isCommand_CommandType (line 573) | func (*Command_StreamingQueryManagerCommand) isCommand_CommandType() {}
type Command_RegisterTableFunction (line 511) | type Command_RegisterTableFunction struct
method isCommand_CommandType (line 575) | func (*Command_RegisterTableFunction) isCommand_CommandType() {}
type Command_StreamingQueryListenerBusCommand (line 515) | type Command_StreamingQueryListenerBusCommand struct
method isCommand_CommandType (line 577) | func (*Command_StreamingQueryListenerBusCommand) isCommand_CommandType...
type Command_RegisterDataSource (line 519) | type Command_RegisterDataSource struct
method isCommand_CommandType (line 579) | func (*Command_RegisterDataSource) isCommand_CommandType() {}
type Command_CreateResourceProfileCommand (line 523) | type Command_CreateResourceProfileCommand struct
method isCommand_CommandType (line 581) | func (*Command_CreateResourceProfileCommand) isCommand_CommandType() {}
type Command_CheckpointCommand (line 527) | type Command_CheckpointCommand struct
method isCommand_CommandType (line 583) | func (*Command_CheckpointCommand) isCommand_CommandType() {}
type Command_RemoveCachedRemoteRelationCommand (line 531) | type Command_RemoveCachedRemoteRelationCommand struct
method isCommand_CommandType (line 585) | func (*Command_RemoveCachedRemoteRelationCommand) isCommand_CommandTyp...
type Command_MergeIntoTableCommand (line 535) | type Command_MergeIntoTableCommand struct
method isCommand_CommandType (line 587) | func (*Command_MergeIntoTableCommand) isCommand_CommandType() {}
type Command_MlCommand (line 539) | type Command_MlCommand struct
method isCommand_CommandType (line 589) | func (*Command_MlCommand) isCommand_CommandType() {}
type Command_ExecuteExternalCommand (line 543) | type Command_ExecuteExternalCommand struct
method isCommand_CommandType (line 591) | func (*Command_ExecuteExternalCommand) isCommand_CommandType() {}
type Command_PipelineCommand (line 547) | type Command_PipelineCommand struct
method isCommand_CommandType (line 593) | func (*Command_PipelineCommand) isCommand_CommandType() {}
type Command_Extension (line 551) | type Command_Extension struct
method isCommand_CommandType (line 595) | func (*Command_Extension) isCommand_CommandType() {}
type SqlCommand (line 603) | type SqlCommand struct
method Reset (line 634) | func (x *SqlCommand) Reset() {
method String (line 643) | func (x *SqlCommand) String() string {
method ProtoMessage (line 647) | func (*SqlCommand) ProtoMessage() {}
method ProtoReflect (line 649) | func (x *SqlCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 662) | func (*SqlCommand) Descriptor() ([]byte, []int) {
method GetSql (line 667) | func (x *SqlCommand) GetSql() string {
method GetArgs (line 675) | func (x *SqlCommand) GetArgs() map[string]*Expression_Literal {
method GetPosArgs (line 683) | func (x *SqlCommand) GetPosArgs() []*Expression_Literal {
method GetNamedArguments (line 691) | func (x *SqlCommand) GetNamedArguments() map[string]*Expression {
method GetPosArguments (line 699) | func (x *SqlCommand) GetPosArguments() []*Expression {
method GetInput (line 706) | func (x *SqlCommand) GetInput() *Relation {
type CreateDataFrameViewCommand (line 714) | type CreateDataFrameViewCommand struct
method Reset (line 732) | func (x *CreateDataFrameViewCommand) Reset() {
method String (line 741) | func (x *CreateDataFrameViewCommand) String() string {
method ProtoMessage (line 745) | func (*CreateDataFrameViewCommand) ProtoMessage() {}
method ProtoReflect (line 747) | func (x *CreateDataFrameViewCommand) ProtoReflect() protoreflect.Messa...
method Descriptor (line 760) | func (*CreateDataFrameViewCommand) Descriptor() ([]byte, []int) {
method GetInput (line 764) | func (x *CreateDataFrameViewCommand) GetInput() *Relation {
method GetName (line 771) | func (x *CreateDataFrameViewCommand) GetName() string {
method GetIsGlobal (line 778) | func (x *CreateDataFrameViewCommand) GetIsGlobal() bool {
method GetReplace (line 785) | func (x *CreateDataFrameViewCommand) GetReplace() bool {
type WriteOperation (line 793) | type WriteOperation struct
method Reset (line 828) | func (x *WriteOperation) Reset() {
method String (line 837) | func (x *WriteOperation) String() string {
method ProtoMessage (line 841) | func (*WriteOperation) ProtoMessage() {}
method ProtoReflect (line 843) | func (x *WriteOperation) ProtoReflect() protoreflect.Message {
method Descriptor (line 856) | func (*WriteOperation) Descriptor() ([]byte, []int) {
method GetInput (line 860) | func (x *WriteOperation) GetInput() *Relation {
method GetSource (line 867) | func (x *WriteOperation) GetSource() string {
method GetSaveType (line 874) | func (m *WriteOperation) GetSaveType() isWriteOperation_SaveType {
method GetPath (line 881) | func (x *WriteOperation) GetPath() string {
method GetTable (line 888) | func (x *WriteOperation) GetTable() *WriteOperation_SaveTable {
method GetMode (line 895) | func (x *WriteOperation) GetMode() WriteOperation_SaveMode {
method GetSortColumnNames (line 902) | func (x *WriteOperation) GetSortColumnNames() []string {
method GetPartitioningColumns (line 909) | func (x *WriteOperation) GetPartitioningColumns() []string {
method GetBucketBy (line 916) | func (x *WriteOperation) GetBucketBy() *WriteOperation_BucketBy {
method GetOptions (line 923) | func (x *WriteOperation) GetOptions() map[string]string {
method GetClusteringColumns (line 930) | func (x *WriteOperation) GetClusteringColumns() []string {
type isWriteOperation_SaveType (line 937) | type isWriteOperation_SaveType interface
type WriteOperation_Path (line 941) | type WriteOperation_Path struct
method isWriteOperation_SaveType (line 949) | func (*WriteOperation_Path) isWriteOperation_SaveType() {}
type WriteOperation_Table (line 945) | type WriteOperation_Table struct
method isWriteOperation_SaveType (line 951) | func (*WriteOperation_Table) isWriteOperation_SaveType() {}
type WriteOperationV2 (line 954) | type WriteOperationV2 struct
method Reset (line 981) | func (x *WriteOperationV2) Reset() {
method String (line 990) | func (x *WriteOperationV2) String() string {
method ProtoMessage (line 994) | func (*WriteOperationV2) ProtoMessage() {}
method ProtoReflect (line 996) | func (x *WriteOperationV2) ProtoReflect() protoreflect.Message {
method Descriptor (line 1009) | func (*WriteOperationV2) Descriptor() ([]byte, []int) {
method GetInput (line 1013) | func (x *WriteOperationV2) GetInput() *Relation {
method GetTableName (line 1020) | func (x *WriteOperationV2) GetTableName() string {
method GetProvider (line 1027) | func (x *WriteOperationV2) GetProvider() string {
method GetPartitioningColumns (line 1034) | func (x *WriteOperationV2) GetPartitioningColumns() []*Expression {
method GetOptions (line 1041) | func (x *WriteOperationV2) GetOptions() map[string]string {
method GetTableProperties (line 1048) | func (x *WriteOperationV2) GetTableProperties() map[string]string {
method GetMode (line 1055) | func (x *WriteOperationV2) GetMode() WriteOperationV2_Mode {
method GetOverwriteCondition (line 1062) | func (x *WriteOperationV2) GetOverwriteCondition() *Expression {
method GetClusteringColumns (line 1069) | func (x *WriteOperationV2) GetClusteringColumns() []string {
type WriteStreamOperationStart (line 1078) | type WriteStreamOperationStart struct
method Reset (line 1110) | func (x *WriteStreamOperationStart) Reset() {
method String (line 1119) | func (x *WriteStreamOperationStart) String() string {
method ProtoMessage (line 1123) | func (*WriteStreamOperationStart) ProtoMessage() {}
method ProtoReflect (line 1125) | func (x *WriteStreamOperationStart) ProtoReflect() protoreflect.Message {
method Descriptor (line 1138) | func (*WriteStreamOperationStart) Descriptor() ([]byte, []int) {
method GetInput (line 1142) | func (x *WriteStreamOperationStart) GetInput() *Relation {
method GetFormat (line 1149) | func (x *WriteStreamOperationStart) GetFormat() string {
method GetOptions (line 1156) | func (x *WriteStreamOperationStart) GetOptions() map[string]string {
method GetPartitioningColumnNames (line 1163) | func (x *WriteStreamOperationStart) GetPartitioningColumnNames() []str...
method GetTrigger (line 1170) | func (m *WriteStreamOperationStart) GetTrigger() isWriteStreamOperatio...
method GetProcessingTimeInterval (line 1177) | func (x *WriteStreamOperationStart) GetProcessingTimeInterval() string {
method GetAvailableNow (line 1184) | func (x *WriteStreamOperationStart) GetAvailableNow() bool {
method GetOnce (line 1191) | func (x *WriteStreamOperationStart) GetOnce() bool {
method GetContinuousCheckpointInterval (line 1198) | func (x *WriteStreamOperationStart) GetContinuousCheckpointInterval() ...
method GetOutputMode (line 1205) | func (x *WriteStreamOperationStart) GetOutputMode() string {
method GetQueryName (line 1212) | func (x *WriteStreamOperationStart) GetQueryName() string {
method GetSinkDestination (line 1219) | func (m *WriteStreamOperationStart) GetSinkDestination() isWriteStream...
method GetPath (line 1226) | func (x *WriteStreamOperationStart) GetPath() string {
method GetTableName (line 1233) | func (x *WriteStreamOperationStart) GetTableName() string {
method GetForeachWriter (line 1240) | func (x *WriteStreamOperationStart) GetForeachWriter() *StreamingForea...
method GetForeachBatch (line 1247) | func (x *WriteStreamOperationStart) GetForeachBatch() *StreamingForeac...
method GetClusteringColumnNames (line 1254) | func (x *WriteStreamOperationStart) GetClusteringColumnNames() []string {
type isWriteStreamOperationStart_Trigger (line 1261) | type isWriteStreamOperationStart_Trigger interface
type WriteStreamOperationStart_ProcessingTimeInterval (line 1265) | type WriteStreamOperationStart_ProcessingTimeInterval struct
method isWriteStreamOperationStart_Trigger (line 1281) | func (*WriteStreamOperationStart_ProcessingTimeInterval) isWriteStream...
type WriteStreamOperationStart_AvailableNow (line 1269) | type WriteStreamOperationStart_AvailableNow struct
method isWriteStreamOperationStart_Trigger (line 1283) | func (*WriteStreamOperationStart_AvailableNow) isWriteStreamOperationS...
type WriteStreamOperationStart_Once (line 1273) | type WriteStreamOperationStart_Once struct
method isWriteStreamOperationStart_Trigger (line 1285) | func (*WriteStreamOperationStart_Once) isWriteStreamOperationStart_Tri...
type WriteStreamOperationStart_ContinuousCheckpointInterval (line 1277) | type WriteStreamOperationStart_ContinuousCheckpointInterval struct
method isWriteStreamOperationStart_Trigger (line 1287) | func (*WriteStreamOperationStart_ContinuousCheckpointInterval) isWrite...
type isWriteStreamOperationStart_SinkDestination (line 1290) | type isWriteStreamOperationStart_SinkDestination interface
type WriteStreamOperationStart_Path (line 1294) | type WriteStreamOperationStart_Path struct
method isWriteStreamOperationStart_SinkDestination (line 1302) | func (*WriteStreamOperationStart_Path) isWriteStreamOperationStart_Sin...
type WriteStreamOperationStart_TableName (line 1298) | type WriteStreamOperationStart_TableName struct
method isWriteStreamOperationStart_SinkDestination (line 1304) | func (*WriteStreamOperationStart_TableName) isWriteStreamOperationStar...
type StreamingForeachFunction (line 1306) | type StreamingForeachFunction struct
method Reset (line 1318) | func (x *StreamingForeachFunction) Reset() {
method String (line 1327) | func (x *StreamingForeachFunction) String() string {
method ProtoMessage (line 1331) | func (*StreamingForeachFunction) ProtoMessage() {}
method ProtoReflect (line 1333) | func (x *StreamingForeachFunction) ProtoReflect() protoreflect.Message {
method Descriptor (line 1346) | func (*StreamingForeachFunction) Descriptor() ([]byte, []int) {
method GetFunction (line 1350) | func (m *StreamingForeachFunction) GetFunction() isStreamingForeachFun...
method GetPythonFunction (line 1357) | func (x *StreamingForeachFunction) GetPythonFunction() *PythonUDF {
method GetScalaFunction (line 1364) | func (x *StreamingForeachFunction) GetScalaFunction() *ScalarScalaUDF {
type isStreamingForeachFunction_Function (line 1371) | type isStreamingForeachFunction_Function interface
type StreamingForeachFunction_PythonFunction (line 1375) | type StreamingForeachFunction_PythonFunction struct
method isStreamingForeachFunction_Function (line 1383) | func (*StreamingForeachFunction_PythonFunction) isStreamingForeachFunc...
type StreamingForeachFunction_ScalaFunction (line 1379) | type StreamingForeachFunction_ScalaFunction struct
method isStreamingForeachFunction_Function (line 1385) | func (*StreamingForeachFunction_ScalaFunction) isStreamingForeachFunct...
type WriteStreamOperationStartResult (line 1387) | type WriteStreamOperationStartResult struct
method Reset (line 1400) | func (x *WriteStreamOperationStartResult) Reset() {
method String (line 1409) | func (x *WriteStreamOperationStartResult) String() string {
method ProtoMessage (line 1413) | func (*WriteStreamOperationStartResult) ProtoMessage() {}
method ProtoReflect (line 1415) | func (x *WriteStreamOperationStartResult) ProtoReflect() protoreflect....
method Descriptor (line 1428) | func (*WriteStreamOperationStartResult) Descriptor() ([]byte, []int) {
method GetQueryId (line 1432) | func (x *WriteStreamOperationStartResult) GetQueryId() *StreamingQuery...
method GetName (line 1439) | func (x *WriteStreamOperationStartResult) GetName() string {
method GetQueryStartedEventJson (line 1446) | func (x *WriteStreamOperationStartResult) GetQueryStartedEventJson() s...
type StreamingQueryInstanceId (line 1456) | type StreamingQueryInstanceId struct
method Reset (line 1471) | func (x *StreamingQueryInstanceId) Reset() {
method String (line 1480) | func (x *StreamingQueryInstanceId) String() string {
method ProtoMessage (line 1484) | func (*StreamingQueryInstanceId) ProtoMessage() {}
method ProtoReflect (line 1486) | func (x *StreamingQueryInstanceId) ProtoReflect() protoreflect.Message {
method Descriptor (line 1499) | func (*StreamingQueryInstanceId) Descriptor() ([]byte, []int) {
method GetId (line 1503) | func (x *StreamingQueryInstanceId) GetId() string {
method GetRunId (line 1510) | func (x *StreamingQueryInstanceId) GetRunId() string {
type StreamingQueryCommand (line 1518) | type StreamingQueryCommand struct
method Reset (line 1540) | func (x *StreamingQueryCommand) Reset() {
method String (line 1549) | func (x *StreamingQueryCommand) String() string {
method ProtoMessage (line 1553) | func (*StreamingQueryCommand) ProtoMessage() {}
method ProtoReflect (line 1555) | func (x *StreamingQueryCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 1568) | func (*StreamingQueryCommand) Descriptor() ([]byte, []int) {
method GetQueryId (line 1572) | func (x *StreamingQueryCommand) GetQueryId() *StreamingQueryInstanceId {
method GetCommand (line 1579) | func (m *StreamingQueryCommand) GetCommand() isStreamingQueryCommand_C...
method GetStatus (line 1586) | func (x *StreamingQueryCommand) GetStatus() bool {
method GetLastProgress (line 1593) | func (x *StreamingQueryCommand) GetLastProgress() bool {
method GetRecentProgress (line 1600) | func (x *StreamingQueryCommand) GetRecentProgress() bool {
method GetStop (line 1607) | func (x *StreamingQueryCommand) GetStop() bool {
method GetProcessAllAvailable (line 1614) | func (x *StreamingQueryCommand) GetProcessAllAvailable() bool {
method GetExplain (line 1621) | func (x *StreamingQueryCommand) GetExplain() *StreamingQueryCommand_Ex...
method GetException (line 1628) | func (x *StreamingQueryCommand) GetException() bool {
method GetAwaitTermination (line 1635) | func (x *StreamingQueryCommand) GetAwaitTermination() *StreamingQueryC...
type isStreamingQueryCommand_Command (line 1642) | type isStreamingQueryCommand_Command interface
type StreamingQueryCommand_Status (line 1646) | type StreamingQueryCommand_Status struct
method isStreamingQueryCommand_Command (line 1686) | func (*StreamingQueryCommand_Status) isStreamingQueryCommand_Command() {}
type StreamingQueryCommand_LastProgress (line 1651) | type StreamingQueryCommand_LastProgress struct
method isStreamingQueryCommand_Command (line 1688) | func (*StreamingQueryCommand_LastProgress) isStreamingQueryCommand_Com...
type StreamingQueryCommand_RecentProgress (line 1656) | type StreamingQueryCommand_RecentProgress struct
method isStreamingQueryCommand_Command (line 1690) | func (*StreamingQueryCommand_RecentProgress) isStreamingQueryCommand_C...
type StreamingQueryCommand_Stop (line 1661) | type StreamingQueryCommand_Stop struct
method isStreamingQueryCommand_Command (line 1692) | func (*StreamingQueryCommand_Stop) isStreamingQueryCommand_Command() {}
type StreamingQueryCommand_ProcessAllAvailable (line 1666) | type StreamingQueryCommand_ProcessAllAvailable struct
method isStreamingQueryCommand_Command (line 1694) | func (*StreamingQueryCommand_ProcessAllAvailable) isStreamingQueryComm...
type StreamingQueryCommand_Explain (line 1671) | type StreamingQueryCommand_Explain struct
method isStreamingQueryCommand_Command (line 1696) | func (*StreamingQueryCommand_Explain) isStreamingQueryCommand_Command(...
type StreamingQueryCommand_Exception (line 1676) | type StreamingQueryCommand_Exception struct
method isStreamingQueryCommand_Command (line 1698) | func (*StreamingQueryCommand_Exception) isStreamingQueryCommand_Comman...
type StreamingQueryCommand_AwaitTermination (line 1681) | type StreamingQueryCommand_AwaitTermination struct
method isStreamingQueryCommand_Command (line 1700) | func (*StreamingQueryCommand_AwaitTermination) isStreamingQueryCommand...
type StreamingQueryCommandResult (line 1703) | type StreamingQueryCommandResult struct
method Reset (line 1720) | func (x *StreamingQueryCommandResult) Reset() {
method String (line 1729) | func (x *StreamingQueryCommandResult) String() string {
method ProtoMessage (line 1733) | func (*StreamingQueryCommandResult) ProtoMessage() {}
method ProtoReflect (line 1735) | func (x *StreamingQueryCommandResult) ProtoReflect() protoreflect.Mess...
method Descriptor (line 1748) | func (*StreamingQueryCommandResult) Descriptor() ([]byte, []int) {
method GetQueryId (line 1752) | func (x *StreamingQueryCommandResult) GetQueryId() *StreamingQueryInst...
method GetResultType (line 1759) | func (m *StreamingQueryCommandResult) GetResultType() isStreamingQuery...
method GetStatus (line 1766) | func (x *StreamingQueryCommandResult) GetStatus() *StreamingQueryComma...
method GetRecentProgress (line 1773) | func (x *StreamingQueryCommandResult) GetRecentProgress() *StreamingQu...
method GetExplain (line 1780) | func (x *StreamingQueryCommandResult) GetExplain() *StreamingQueryComm...
method GetException (line 1787) | func (x *StreamingQueryCommandResult) GetException() *StreamingQueryCo...
method GetAwaitTermination (line 1794) | func (x *StreamingQueryCommandResult) GetAwaitTermination() *Streaming...
type isStreamingQueryCommandResult_ResultType (line 1801) | type isStreamingQueryCommandResult_ResultType interface
type StreamingQueryCommandResult_Status (line 1805) | type StreamingQueryCommandResult_Status struct
method isStreamingQueryCommandResult_ResultType (line 1825) | func (*StreamingQueryCommandResult_Status) isStreamingQueryCommandResu...
type StreamingQueryCommandResult_RecentProgress (line 1809) | type StreamingQueryCommandResult_RecentProgress struct
method isStreamingQueryCommandResult_ResultType (line 1827) | func (*StreamingQueryCommandResult_RecentProgress) isStreamingQueryCom...
type StreamingQueryCommandResult_Explain (line 1813) | type StreamingQueryCommandResult_Explain struct
method isStreamingQueryCommandResult_ResultType (line 1829) | func (*StreamingQueryCommandResult_Explain) isStreamingQueryCommandRes...
type StreamingQueryCommandResult_Exception (line 1817) | type StreamingQueryCommandResult_Exception struct
method isStreamingQueryCommandResult_ResultType (line 1831) | func (*StreamingQueryCommandResult_Exception) isStreamingQueryCommandR...
type StreamingQueryCommandResult_AwaitTermination (line 1821) | type StreamingQueryCommandResult_AwaitTermination struct
method isStreamingQueryCommandResult_ResultType (line 1833) | func (*StreamingQueryCommandResult_AwaitTermination) isStreamingQueryC...
type StreamingQueryManagerCommand (line 1836) | type StreamingQueryManagerCommand struct
method Reset (line 1855) | func (x *StreamingQueryManagerCommand) Reset() {
method String (line 1864) | func (x *StreamingQueryManagerCommand) String() string {
method ProtoMessage (line 1868) | func (*StreamingQueryManagerCommand) ProtoMessage() {}
method ProtoReflect (line 1870) | func (x *StreamingQueryManagerCommand) ProtoReflect() protoreflect.Mes...
method Descriptor (line 1883) | func (*StreamingQueryManagerCommand) Descriptor() ([]byte, []int) {
method GetCommand (line 1887) | func (m *StreamingQueryManagerCommand) GetCommand() isStreamingQueryMa...
method GetActive (line 1894) | func (x *StreamingQueryManagerCommand) GetActive() bool {
method GetGetQuery (line 1901) | func (x *StreamingQueryManagerCommand) GetGetQuery() string {
method GetAwaitAnyTermination (line 1908) | func (x *StreamingQueryManagerCommand) GetAwaitAnyTermination() *Strea...
method GetResetTerminated (line 1915) | func (x *StreamingQueryManagerCommand) GetResetTerminated() bool {
method GetAddListener (line 1922) | func (x *StreamingQueryManagerCommand) GetAddListener() *StreamingQuer...
method GetRemoveListener (line 1929) | func (x *StreamingQueryManagerCommand) GetRemoveListener() *StreamingQ...
method GetListListeners (line 1936) | func (x *StreamingQueryManagerCommand) GetListListeners() bool {
type isStreamingQueryManagerCommand_Command (line 1943) | type isStreamingQueryManagerCommand_Command interface
type StreamingQueryManagerCommand_Active (line 1947) | type StreamingQueryManagerCommand_Active struct
method isStreamingQueryManagerCommand_Command (line 1982) | func (*StreamingQueryManagerCommand_Active) isStreamingQueryManagerCom...
type StreamingQueryManagerCommand_GetQuery (line 1952) | type StreamingQueryManagerCommand_GetQuery struct
method isStreamingQueryManagerCommand_Command (line 1984) | func (*StreamingQueryManagerCommand_GetQuery) isStreamingQueryManagerC...
type StreamingQueryManagerCommand_AwaitAnyTermination (line 1957) | type StreamingQueryManagerCommand_AwaitAnyTermination struct
method isStreamingQueryManagerCommand_Command (line 1986) | func (*StreamingQueryManagerCommand_AwaitAnyTermination) isStreamingQu...
type StreamingQueryManagerCommand_ResetTerminated (line 1962) | type StreamingQueryManagerCommand_ResetTerminated struct
method isStreamingQueryManagerCommand_Command (line 1988) | func (*StreamingQueryManagerCommand_ResetTerminated) isStreamingQueryM...
type StreamingQueryManagerCommand_AddListener (line 1967) | type StreamingQueryManagerCommand_AddListener struct
method isStreamingQueryManagerCommand_Command (line 1990) | func (*StreamingQueryManagerCommand_AddListener) isStreamingQueryManag...
type StreamingQueryManagerCommand_RemoveListener (line 1972) | type StreamingQueryManagerCommand_RemoveListener struct
method isStreamingQueryManagerCommand_Command (line 1992) | func (*StreamingQueryManagerCommand_RemoveListener) isStreamingQueryMa...
type StreamingQueryManagerCommand_ListListeners (line 1977) | type StreamingQueryManagerCommand_ListListeners struct
method isStreamingQueryManagerCommand_Command (line 1994) | func (*StreamingQueryManagerCommand_ListListeners) isStreamingQueryMan...
type StreamingQueryManagerCommandResult (line 1997) | type StreamingQueryManagerCommandResult struct
method Reset (line 2014) | func (x *StreamingQueryManagerCommandResult) Reset() {
method String (line 2023) | func (x *StreamingQueryManagerCommandResult) String() string {
method ProtoMessage (line 2027) | func (*StreamingQueryManagerCommandResult) ProtoMessage() {}
method ProtoReflect (line 2029) | func (x *StreamingQueryManagerCommandResult) ProtoReflect() protorefle...
method Descriptor (line 2042) | func (*StreamingQueryManagerCommandResult) Descriptor() ([]byte, []int) {
method GetResultType (line 2046) | func (m *StreamingQueryManagerCommandResult) GetResultType() isStreami...
method GetActive (line 2053) | func (x *StreamingQueryManagerCommandResult) GetActive() *StreamingQue...
method GetQuery (line 2060) | func (x *StreamingQueryManagerCommandResult) GetQuery() *StreamingQuer...
method GetAwaitAnyTermination (line 2067) | func (x *StreamingQueryManagerCommandResult) GetAwaitAnyTermination() ...
method GetResetTerminated (line 2074) | func (x *StreamingQueryManagerCommandResult) GetResetTerminated() bool {
method GetAddListener (line 2081) | func (x *StreamingQueryManagerCommandResult) GetAddListener() bool {
method GetRemoveListener (line 2088) | func (x *StreamingQueryManagerCommandResult) GetRemoveListener() bool {
method GetListListeners (line 2095) | func (x *StreamingQueryManagerCommandResult) GetListListeners() *Strea...
type isStreamingQueryManagerCommandResult_ResultType (line 2102) | type isStreamingQueryManagerCommandResult_ResultType interface
type StreamingQueryManagerCommandResult_Active (line 2106) | type StreamingQueryManagerCommandResult_Active struct
method isStreamingQueryManagerCommandResult_ResultType (line 2134) | func (*StreamingQueryManagerCommandResult_Active) isStreamingQueryMana...
type StreamingQueryManagerCommandResult_Query (line 2110) | type StreamingQueryManagerCommandResult_Query struct
method isStreamingQueryManagerCommandResult_ResultType (line 2136) | func (*StreamingQueryManagerCommandResult_Query) isStreamingQueryManag...
type StreamingQueryManagerCommandResult_AwaitAnyTermination (line 2114) | type StreamingQueryManagerCommandResult_AwaitAnyTermination struct
method isStreamingQueryManagerCommandResult_ResultType (line 2138) | func (*StreamingQueryManagerCommandResult_AwaitAnyTermination) isStrea...
type StreamingQueryManagerCommandResult_ResetTerminated (line 2118) | type StreamingQueryManagerCommandResult_ResetTerminated struct
method isStreamingQueryManagerCommandResult_ResultType (line 2141) | func (*StreamingQueryManagerCommandResult_ResetTerminated) isStreaming...
type StreamingQueryManagerCommandResult_AddListener (line 2122) | type StreamingQueryManagerCommandResult_AddListener struct
method isStreamingQueryManagerCommandResult_ResultType (line 2144) | func (*StreamingQueryManagerCommandResult_AddListener) isStreamingQuer...
type StreamingQueryManagerCommandResult_RemoveListener (line 2126) | type StreamingQueryManagerCommandResult_RemoveListener struct
method isStreamingQueryManagerCommandResult_ResultType (line 2147) | func (*StreamingQueryManagerCommandResult_RemoveListener) isStreamingQ...
type StreamingQueryManagerCommandResult_ListListeners (line 2130) | type StreamingQueryManagerCommandResult_ListListeners struct
method isStreamingQueryManagerCommandResult_ResultType (line 2150) | func (*StreamingQueryManagerCommandResult_ListListeners) isStreamingQu...
type StreamingQueryListenerBusCommand (line 2158) | type StreamingQueryListenerBusCommand struct
method Reset (line 2170) | func (x *StreamingQueryListenerBusCommand) Reset() {
method String (line 2179) | func (x *StreamingQueryListenerBusCommand) String() string {
method ProtoMessage (line 2183) | func (*StreamingQueryListenerBusCommand) ProtoMessage() {}
method ProtoReflect (line 2185) | func (x *StreamingQueryListenerBusCommand) ProtoReflect() protoreflect...
method Descriptor (line 2198) | func (*StreamingQueryListenerBusCommand) Descriptor() ([]byte, []int) {
method GetCommand (line 2202) | func (m *StreamingQueryListenerBusCommand) GetCommand() isStreamingQue...
method GetAddListenerBusListener (line 2209) | func (x *StreamingQueryListenerBusCommand) GetAddListenerBusListener()...
method GetRemoveListenerBusListener (line 2216) | func (x *StreamingQueryListenerBusCommand) GetRemoveListenerBusListene...
type isStreamingQueryListenerBusCommand_Command (line 2223) | type isStreamingQueryListenerBusCommand_Command interface
type StreamingQueryListenerBusCommand_AddListenerBusListener (line 2227) | type StreamingQueryListenerBusCommand_AddListenerBusListener struct
method isStreamingQueryListenerBusCommand_Command (line 2235) | func (*StreamingQueryListenerBusCommand_AddListenerBusListener) isStre...
type StreamingQueryListenerBusCommand_RemoveListenerBusListener (line 2231) | type StreamingQueryListenerBusCommand_RemoveListenerBusListener struct
method isStreamingQueryListenerBusCommand_Command (line 2238) | func (*StreamingQueryListenerBusCommand_RemoveListenerBusListener) isS...
type StreamingQueryListenerEvent (line 2242) | type StreamingQueryListenerEvent struct
method Reset (line 2253) | func (x *StreamingQueryListenerEvent) Reset() {
method String (line 2262) | func (x *StreamingQueryListenerEvent) String() string {
method ProtoMessage (line 2266) | func (*StreamingQueryListenerEvent) ProtoMessage() {}
method ProtoReflect (line 2268) | func (x *StreamingQueryListenerEvent) ProtoReflect() protoreflect.Mess...
method Descriptor (line 2281) | func (*StreamingQueryListenerEvent) Descriptor() ([]byte, []int) {
method GetEventJson (line 2285) | func (x *StreamingQueryListenerEvent) GetEventJson() string {
method GetEventType (line 2292) | func (x *StreamingQueryListenerEvent) GetEventType() StreamingQueryEve...
type StreamingQueryListenerEventsResult (line 2299) | type StreamingQueryListenerEventsResult struct
method Reset (line 2308) | func (x *StreamingQueryListenerEventsResult) Reset() {
method String (line 2317) | func (x *StreamingQueryListenerEventsResult) String() string {
method ProtoMessage (line 2321) | func (*StreamingQueryListenerEventsResult) ProtoMessage() {}
method ProtoReflect (line 2323) | func (x *StreamingQueryListenerEventsResult) ProtoReflect() protorefle...
method Descriptor (line 2336) | func (*StreamingQueryListenerEventsResult) Descriptor() ([]byte, []int) {
method GetEvents (line 2340) | func (x *StreamingQueryListenerEventsResult) GetEvents() []*StreamingQ...
method GetListenerBusListenerAdded (line 2347) | func (x *StreamingQueryListenerEventsResult) GetListenerBusListenerAdd...
type GetResourcesCommand (line 2355) | type GetResourcesCommand struct
method Reset (line 2361) | func (x *GetResourcesCommand) Reset() {
method String (line 2370) | func (x *GetResourcesCommand) String() string {
method ProtoMessage (line 2374) | func (*GetResourcesCommand) ProtoMessage() {}
method ProtoReflect (line 2376) | func (x *GetResourcesCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 2389) | func (*GetResourcesCommand) Descriptor() ([]byte, []int) {
type GetResourcesCommandResult (line 2394) | type GetResourcesCommandResult struct
method Reset (line 2402) | func (x *GetResourcesCommandResult) Reset() {
method String (line 2411) | func (x *GetResourcesCommandResult) String() string {
method ProtoMessage (line 2415) | func (*GetResourcesCommandResult) ProtoMessage() {}
method ProtoReflect (line 2417) | func (x *GetResourcesCommandResult) ProtoReflect() protoreflect.Message {
method Descriptor (line 2430) | func (*GetResourcesCommandResult) Descriptor() ([]byte, []int) {
method GetResources (line 2434) | func (x *GetResourcesCommandResult) GetResources() map[string]*Resourc...
type CreateResourceProfileCommand (line 2442) | type CreateResourceProfileCommand struct
method Reset (line 2451) | func (x *CreateResourceProfileCommand) Reset() {
method String (line 2460) | func (x *CreateResourceProfileCommand) String() string {
method ProtoMessage (line 2464) | func (*CreateResourceProfileCommand) ProtoMessage() {}
method ProtoReflect (line 2466) | func (x *CreateResourceProfileCommand) ProtoReflect() protoreflect.Mes...
method Descriptor (line 2479) | func (*CreateResourceProfileCommand) Descriptor() ([]byte, []int) {
method GetProfile (line 2483) | func (x *CreateResourceProfileCommand) GetProfile() *ResourceProfile {
type CreateResourceProfileCommandResult (line 2491) | type CreateResourceProfileCommandResult struct
method Reset (line 2500) | func (x *CreateResourceProfileCommandResult) Reset() {
method String (line 2509) | func (x *CreateResourceProfileCommandResult) String() string {
method ProtoMessage (line 2513) | func (*CreateResourceProfileCommandResult) ProtoMessage() {}
method ProtoReflect (line 2515) | func (x *CreateResourceProfileCommandResult) ProtoReflect() protorefle...
method Descriptor (line 2528) | func (*CreateResourceProfileCommandResult) Descriptor() ([]byte, []int) {
method GetProfileId (line 2532) | func (x *CreateResourceProfileCommandResult) GetProfileId() int32 {
type RemoveCachedRemoteRelationCommand (line 2540) | type RemoveCachedRemoteRelationCommand struct
method Reset (line 2549) | func (x *RemoveCachedRemoteRelationCommand) Reset() {
method String (line 2558) | func (x *RemoveCachedRemoteRelationCommand) String() string {
method ProtoMessage (line 2562) | func (*RemoveCachedRemoteRelationCommand) ProtoMessage() {}
method ProtoReflect (line 2564) | func (x *RemoveCachedRemoteRelationCommand) ProtoReflect() protoreflec...
method Descriptor (line 2577) | func (*RemoveCachedRemoteRelationCommand) Descriptor() ([]byte, []int) {
method GetRelation (line 2581) | func (x *RemoveCachedRemoteRelationCommand) GetRelation() *CachedRemot...
type CheckpointCommand (line 2588) | type CheckpointCommand struct
method Reset (line 2604) | func (x *CheckpointCommand) Reset() {
method String (line 2613) | func (x *CheckpointCommand) String() string {
method ProtoMessage (line 2617) | func (*CheckpointCommand) ProtoMessage() {}
method ProtoReflect (line 2619) | func (x *CheckpointCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 2632) | func (*CheckpointCommand) Descriptor() ([]byte, []int) {
method GetRelation (line 2636) | func (x *CheckpointCommand) GetRelation() *Relation {
method GetLocal (line 2643) | func (x *CheckpointCommand) GetLocal() bool {
method GetEager (line 2650) | func (x *CheckpointCommand) GetEager() bool {
method GetStorageLevel (line 2657) | func (x *CheckpointCommand) GetStorageLevel() *StorageLevel {
type MergeIntoTableCommand (line 2664) | type MergeIntoTableCommand struct
method Reset (line 2685) | func (x *MergeIntoTableCommand) Reset() {
method String (line 2694) | func (x *MergeIntoTableCommand) String() string {
method ProtoMessage (line 2698) | func (*MergeIntoTableCommand) ProtoMessage() {}
method ProtoReflect (line 2700) | func (x *MergeIntoTableCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 2713) | func (*MergeIntoTableCommand) Descriptor() ([]byte, []int) {
method GetTargetTableName (line 2717) | func (x *MergeIntoTableCommand) GetTargetTableName() string {
method GetSourceTablePlan (line 2724) | func (x *MergeIntoTableCommand) GetSourceTablePlan() *Relation {
method GetMergeCondition (line 2731) | func (x *MergeIntoTableCommand) GetMergeCondition() *Expression {
method GetMatchActions (line 2738) | func (x *MergeIntoTableCommand) GetMatchActions() []*Expression {
method GetNotMatchedActions (line 2745) | func (x *MergeIntoTableCommand) GetNotMatchedActions() []*Expression {
method GetNotMatchedBySourceActions (line 2752) | func (x *MergeIntoTableCommand) GetNotMatchedBySourceActions() []*Expr...
method GetWithSchemaEvolution (line 2759) | func (x *MergeIntoTableCommand) GetWithSchemaEvolution() bool {
type ExecuteExternalCommand (line 2767) | type ExecuteExternalCommand struct
method Reset (line 2780) | func (x *ExecuteExternalCommand) Reset() {
method String (line 2789) | func (x *ExecuteExternalCommand) String() string {
method ProtoMessage (line 2793) | func (*ExecuteExternalCommand) ProtoMessage() {}
method ProtoReflect (line 2795) | func (x *ExecuteExternalCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 2808) | func (*ExecuteExternalCommand) Descriptor() ([]byte, []int) {
method GetRunner (line 2812) | func (x *ExecuteExternalCommand) GetRunner() string {
method GetCommand (line 2819) | func (x *ExecuteExternalCommand) GetCommand() string {
method GetOptions (line 2826) | func (x *ExecuteExternalCommand) GetOptions() map[string]string {
type WriteOperation_SaveTable (line 2833) | type WriteOperation_SaveTable struct
method Reset (line 2844) | func (x *WriteOperation_SaveTable) Reset() {
method String (line 2853) | func (x *WriteOperation_SaveTable) String() string {
method ProtoMessage (line 2857) | func (*WriteOperation_SaveTable) ProtoMessage() {}
method ProtoReflect (line 2859) | func (x *WriteOperation_SaveTable) ProtoReflect() protoreflect.Message {
method Descriptor (line 2872) | func (*WriteOperation_SaveTable) Descriptor() ([]byte, []int) {
method GetTableName (line 2876) | func (x *WriteOperation_SaveTable) GetTableName() string {
method GetSaveMethod (line 2883) | func (x *WriteOperation_SaveTable) GetSaveMethod() WriteOperation_Save...
type WriteOperation_BucketBy (line 2890) | type WriteOperation_BucketBy struct
method Reset (line 2899) | func (x *WriteOperation_BucketBy) Reset() {
method String (line 2908) | func (x *WriteOperation_BucketBy) String() string {
method ProtoMessage (line 2912) | func (*WriteOperation_BucketBy) ProtoMessage() {}
method ProtoReflect (line 2914) | func (x *WriteOperation_BucketBy) ProtoReflect() protoreflect.Message {
method Descriptor (line 2927) | func (*WriteOperation_BucketBy) Descriptor() ([]byte, []int) {
method GetBucketColumnNames (line 2931) | func (x *WriteOperation_BucketBy) GetBucketColumnNames() []string {
method GetNumBuckets (line 2938) | func (x *WriteOperation_BucketBy) GetNumBuckets() int32 {
type StreamingQueryCommand_ExplainCommand (line 2945) | type StreamingQueryCommand_ExplainCommand struct
method Reset (line 2956) | func (x *StreamingQueryCommand_ExplainCommand) Reset() {
method String (line 2965) | func (x *StreamingQueryCommand_ExplainCommand) String() string {
method ProtoMessage (line 2969) | func (*StreamingQueryCommand_ExplainCommand) ProtoMessage() {}
method ProtoReflect (line 2971) | func (x *StreamingQueryCommand_ExplainCommand) ProtoReflect() protoref...
method Descriptor (line 2984) | func (*StreamingQueryCommand_ExplainCommand) Descriptor() ([]byte, []i...
method GetExtended (line 2988) | func (x *StreamingQueryCommand_ExplainCommand) GetExtended() bool {
type StreamingQueryCommand_AwaitTerminationCommand (line 2995) | type StreamingQueryCommand_AwaitTerminationCommand struct
method Reset (line 3003) | func (x *StreamingQueryCommand_AwaitTerminationCommand) Reset() {
method String (line 3012) | func (x *StreamingQueryCommand_AwaitTerminationCommand) String() string {
method ProtoMessage (line 3016) | func (*StreamingQueryCommand_AwaitTerminationCommand) ProtoMessage() {}
method ProtoReflect (line 3018) | func (x *StreamingQueryCommand_AwaitTerminationCommand) ProtoReflect()...
method Descriptor (line 3031) | func (*StreamingQueryCommand_AwaitTerminationCommand) Descriptor() ([]...
method GetTimeoutMs (line 3035) | func (x *StreamingQueryCommand_AwaitTerminationCommand) GetTimeoutMs()...
type StreamingQueryCommandResult_StatusResult (line 3042) | type StreamingQueryCommandResult_StatusResult struct
method Reset (line 3054) | func (x *StreamingQueryCommandResult_StatusResult) Reset() {
method String (line 3063) | func (x *StreamingQueryCommandResult_StatusResult) String() string {
method ProtoMessage (line 3067) | func (*StreamingQueryCommandResult_StatusResult) ProtoMessage() {}
method ProtoReflect (line 3069) | func (x *StreamingQueryCommandResult_StatusResult) ProtoReflect() prot...
method Descriptor (line 3082) | func (*StreamingQueryCommandResult_StatusResult) Descriptor() ([]byte,...
method GetStatusMessage (line 3086) | func (x *StreamingQueryCommandResult_StatusResult) GetStatusMessage() ...
method GetIsDataAvailable (line 3093) | func (x *StreamingQueryCommandResult_StatusResult) GetIsDataAvailable(...
method GetIsTriggerActive (line 3100) | func (x *StreamingQueryCommandResult_StatusResult) GetIsTriggerActive(...
method GetIsActive (line 3107) | func (x *StreamingQueryCommandResult_StatusResult) GetIsActive() bool {
type StreamingQueryCommandResult_RecentProgressResult (line 3114) | type StreamingQueryCommandResult_RecentProgressResult struct
method Reset (line 3123) | func (x *StreamingQueryCommandResult_RecentProgressResult) Reset() {
method String (line 3132) | func (x *StreamingQueryCommandResult_RecentProgressResult) String() st...
method ProtoMessage (line 3136) | func (*StreamingQueryCommandResult_RecentProgressResult) ProtoMessage(...
method ProtoReflect (line 3138) | func (x *StreamingQueryCommandResult_RecentProgressResult) ProtoReflec...
method Descriptor (line 3151) | func (*StreamingQueryCommandResult_RecentProgressResult) Descriptor() ...
method GetRecentProgressJson (line 3155) | func (x *StreamingQueryCommandResult_RecentProgressResult) GetRecentPr...
type StreamingQueryCommandResult_ExplainResult (line 3162) | type StreamingQueryCommandResult_ExplainResult struct
method Reset (line 3171) | func (x *StreamingQueryCommandResult_ExplainResult) Reset() {
method String (line 3180) | func (x *StreamingQueryCommandResult_ExplainResult) String() string {
method ProtoMessage (line 3184) | func (*StreamingQueryCommandResult_ExplainResult) ProtoMessage() {}
method ProtoReflect (line 3186) | func (x *StreamingQueryCommandResult_ExplainResult) ProtoReflect() pro...
method Descriptor (line 3199) | func (*StreamingQueryCommandResult_ExplainResult) Descriptor() ([]byte...
method GetResult (line 3203) | func (x *StreamingQueryCommandResult_ExplainResult) GetResult() string {
type StreamingQueryCommandResult_ExceptionResult (line 3210) | type StreamingQueryCommandResult_ExceptionResult struct
method Reset (line 3224) | func (x *StreamingQueryCommandResult_ExceptionResult) Reset() {
method String (line 3233) | func (x *StreamingQueryCommandResult_ExceptionResult) String() string {
method ProtoMessage (line 3237) | func (*StreamingQueryCommandResult_ExceptionResult) ProtoMessage() {}
method ProtoReflect (line 3239) | func (x *StreamingQueryCommandResult_ExceptionResult) ProtoReflect() p...
method Descriptor (line 3252) | func (*StreamingQueryCommandResult_ExceptionResult) Descriptor() ([]by...
method GetExceptionMessage (line 3256) | func (x *StreamingQueryCommandResult_ExceptionResult) GetExceptionMess...
method GetErrorClass (line 3263) | func (x *StreamingQueryCommandResult_ExceptionResult) GetErrorClass() ...
method GetStackTrace (line 3270) | func (x *StreamingQueryCommandResult_ExceptionResult) GetStackTrace() ...
type StreamingQueryCommandResult_AwaitTerminationResult (line 3277) | type StreamingQueryCommandResult_AwaitTerminationResult struct
method Reset (line 3285) | func (x *StreamingQueryCommandResult_AwaitTerminationResult) Reset() {
method String (line 3294) | func (x *StreamingQueryCommandResult_AwaitTerminationResult) String() ...
method ProtoMessage (line 3298) | func (*StreamingQueryCommandResult_AwaitTerminationResult) ProtoMessag...
method ProtoReflect (line 3300) | func (x *StreamingQueryCommandResult_AwaitTerminationResult) ProtoRefl...
method Descriptor (line 3313) | func (*StreamingQueryCommandResult_AwaitTerminationResult) Descriptor(...
method GetTerminated (line 3317) | func (x *StreamingQueryCommandResult_AwaitTerminationResult) GetTermin...
type StreamingQueryManagerCommand_AwaitAnyTerminationCommand (line 3324) | type StreamingQueryManagerCommand_AwaitAnyTerminationCommand struct
method Reset (line 3333) | func (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Rese...
method String (line 3342) | func (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Stri...
method ProtoMessage (line 3346) | func (*StreamingQueryManagerCommand_AwaitAnyTerminationCommand) ProtoM...
method ProtoReflect (line 3348) | func (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Prot...
method Descriptor (line 3361) | func (*StreamingQueryManagerCommand_AwaitAnyTerminationCommand) Descri...
method GetTimeoutMs (line 3365) | func (x *StreamingQueryManagerCommand_AwaitAnyTerminationCommand) GetT...
type StreamingQueryManagerCommand_StreamingQueryListenerCommand (line 3372) | type StreamingQueryManagerCommand_StreamingQueryListenerCommand struct
method Reset (line 3382) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) R...
method String (line 3391) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) S...
method ProtoMessage (line 3395) | func (*StreamingQueryManagerCommand_StreamingQueryListenerCommand) Pro...
method ProtoReflect (line 3397) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) P...
method Descriptor (line 3410) | func (*StreamingQueryManagerCommand_StreamingQueryListenerCommand) Des...
method GetListenerPayload (line 3414) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) G...
method GetPythonListenerPayload (line 3421) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) G...
method GetId (line 3428) | func (x *StreamingQueryManagerCommand_StreamingQueryListenerCommand) G...
type StreamingQueryManagerCommandResult_ActiveResult (line 3435) | type StreamingQueryManagerCommandResult_ActiveResult struct
method Reset (line 3443) | func (x *StreamingQueryManagerCommandResult_ActiveResult) Reset() {
method String (line 3452) | func (x *StreamingQueryManagerCommandResult_ActiveResult) String() str...
method ProtoMessage (line 3456) | func (*StreamingQueryManagerCommandResult_ActiveResult) ProtoMessage() {}
method ProtoReflect (line 3458) | func (x *StreamingQueryManagerCommandResult_ActiveResult) ProtoReflect...
method Descriptor (line 3471) | func (*StreamingQueryManagerCommandResult_ActiveResult) Descriptor() (...
method GetActiveQueries (line 3475) | func (x *StreamingQueryManagerCommandResult_ActiveResult) GetActiveQue...
type StreamingQueryManagerCommandResult_StreamingQueryInstance (line 3482) | type StreamingQueryManagerCommandResult_StreamingQueryInstance struct
method Reset (line 3493) | func (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) Re...
method String (line 3502) | func (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) St...
method ProtoMessage (line 3506) | func (*StreamingQueryManagerCommandResult_StreamingQueryInstance) Prot...
method ProtoReflect (line 3508) | func (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) Pr...
method Descriptor (line 3521) | func (*StreamingQueryManagerCommandResult_StreamingQueryInstance) Desc...
method GetId (line 3525) | func (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) Ge...
method GetName (line 3532) | func (x *StreamingQueryManagerCommandResult_StreamingQueryInstance) Ge...
type StreamingQueryManagerCommandResult_AwaitAnyTerminationResult (line 3539) | type StreamingQueryManagerCommandResult_AwaitAnyTerminationResult struct
method Reset (line 3547) | func (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult)...
method String (line 3556) | func (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult)...
method ProtoMessage (line 3560) | func (*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) P...
method ProtoReflect (line 3562) | func (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult)...
method Descriptor (line 3575) | func (*StreamingQueryManagerCommandResult_AwaitAnyTerminationResult) D...
method GetTerminated (line 3579) | func (x *StreamingQueryManagerCommandResult_AwaitAnyTerminationResult)...
type StreamingQueryManagerCommandResult_StreamingQueryListenerInstance (line 3586) | type StreamingQueryManagerCommandResult_StreamingQueryListenerInstance s...
method Reset (line 3594) | func (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInst...
method String (line 3603) | func (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInst...
method ProtoMessage (line 3607) | func (*StreamingQueryManagerCommandResult_StreamingQueryListenerInstan...
method ProtoReflect (line 3609) | func (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInst...
method Descriptor (line 3622) | func (*StreamingQueryManagerCommandResult_StreamingQueryListenerInstan...
method GetListenerPayload (line 3626) | func (x *StreamingQueryManagerCommandResult_StreamingQueryListenerInst...
type StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult (line 3633) | type StreamingQueryManagerCommandResult_ListStreamingQueryListenerResult...
method Reset (line 3642) | func (x *StreamingQueryManagerCommandResult_ListStreamingQueryListener...
method String (line 3651) | func (x *StreamingQueryManagerCommandResult_ListStreamingQueryListener...
method ProtoMessage (line 3655) | func (*StreamingQueryManagerCommandResult_ListStreamingQueryListenerRe...
method ProtoReflect (line 3657) | func (x *StreamingQueryManagerCommandResult_ListStreamingQueryListener...
method Descriptor (line 3670) | func (*StreamingQueryManagerCommandResult_ListStreamingQueryListenerRe...
method GetListenerIds (line 3674) | func (x *StreamingQueryManagerCommandResult_ListStreamingQueryListener...
function file_spark_connect_commands_proto_rawDescGZIP (line 4439) | func file_spark_connect_commands_proto_rawDescGZIP() []byte {
function init (line 4605) | func init() { file_spark_connect_commands_proto_init() }
function file_spark_connect_commands_proto_init (line 4606) | func file_spark_connect_commands_proto_init() {
FILE: internal/generated/common.pb.go
constant _ (line 34) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 36) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type StorageLevel (line 40) | type StorageLevel struct
method Reset (line 57) | func (x *StorageLevel) Reset() {
method String (line 66) | func (x *StorageLevel) String() string {
method ProtoMessage (line 70) | func (*StorageLevel) ProtoMessage() {}
method ProtoReflect (line 72) | func (x *StorageLevel) ProtoReflect() protoreflect.Message {
method Descriptor (line 85) | func (*StorageLevel) Descriptor() ([]byte, []int) {
method GetUseDisk (line 89) | func (x *StorageLevel) GetUseDisk() bool {
method GetUseMemory (line 96) | func (x *StorageLevel) GetUseMemory() bool {
method GetUseOffHeap (line 103) | func (x *StorageLevel) GetUseOffHeap() bool {
method GetDeserialized (line 110) | func (x *StorageLevel) GetDeserialized() bool {
method GetReplication (line 117) | func (x *StorageLevel) GetReplication() int32 {
type ResourceInformation (line 126) | type ResourceInformation struct
method Reset (line 137) | func (x *ResourceInformation) Reset() {
method String (line 146) | func (x *ResourceInformation) String() string {
method ProtoMessage (line 150) | func (*ResourceInformation) ProtoMessage() {}
method ProtoReflect (line 152) | func (x *ResourceInformation) ProtoReflect() protoreflect.Message {
method Descriptor (line 165) | func (*ResourceInformation) Descriptor() ([]byte, []int) {
method GetName (line 169) | func (x *ResourceInformation) GetName() string {
method GetAddresses (line 176) | func (x *ResourceInformation) GetAddresses() []string {
type ExecutorResourceRequest (line 184) | type ExecutorResourceRequest struct
method Reset (line 199) | func (x *ExecutorResourceRequest) Reset() {
method String (line 208) | func (x *ExecutorResourceRequest) String() string {
method ProtoMessage (line 212) | func (*ExecutorResourceRequest) ProtoMessage() {}
method ProtoReflect (line 214) | func (x *ExecutorResourceRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 227) | func (*ExecutorResourceRequest) Descriptor() ([]byte, []int) {
method GetResourceName (line 231) | func (x *ExecutorResourceRequest) GetResourceName() string {
method GetAmount (line 238) | func (x *ExecutorResourceRequest) GetAmount() int64 {
method GetDiscoveryScript (line 245) | func (x *ExecutorResourceRequest) GetDiscoveryScript() string {
method GetVendor (line 252) | func (x *ExecutorResourceRequest) GetVendor() string {
type TaskResourceRequest (line 260) | type TaskResourceRequest struct
method Reset (line 272) | func (x *TaskResourceRequest) Reset() {
method String (line 281) | func (x *TaskResourceRequest) String() string {
method ProtoMessage (line 285) | func (*TaskResourceRequest) ProtoMessage() {}
method ProtoReflect (line 287) | func (x *TaskResourceRequest) ProtoReflect() protoreflect.Message {
method Descriptor (line 300) | func (*TaskResourceRequest) Descriptor() ([]byte, []int) {
method GetResourceName (line 304) | func (x *TaskResourceRequest) GetResourceName() string {
method GetAmount (line 311) | func (x *TaskResourceRequest) GetAmount() float64 {
type ResourceProfile (line 318) | type ResourceProfile struct
method Reset (line 331) | func (x *ResourceProfile) Reset() {
method String (line 340) | func (x *ResourceProfile) String() string {
method ProtoMessage (line 344) | func (*ResourceProfile) ProtoMessage() {}
method ProtoReflect (line 346) | func (x *ResourceProfile) ProtoReflect() protoreflect.Message {
method Descriptor (line 359) | func (*ResourceProfile) Descriptor() ([]byte, []int) {
method GetExecutorResources (line 363) | func (x *ResourceProfile) GetExecutorResources() map[string]*ExecutorR...
method GetTaskResources (line 370) | func (x *ResourceProfile) GetTaskResources() map[string]*TaskResourceR...
type Origin (line 377) | type Origin struct
method Reset (line 391) | func (x *Origin) Reset() {
method String (line 400) | func (x *Origin) String() string {
method ProtoMessage (line 404) | func (*Origin) ProtoMessage() {}
method ProtoReflect (line 406) | func (x *Origin) ProtoReflect() protoreflect.Message {
method Descriptor (line 419) | func (*Origin) Descriptor() ([]byte, []int) {
method GetFunction (line 423) | func (m *Origin) GetFunction() isOrigin_Function {
method GetPythonOrigin (line 430) | func (x *Origin) GetPythonOrigin() *PythonOrigin {
method GetJvmOrigin (line 437) | func (x *Origin) GetJvmOrigin() *JvmOrigin {
type isOrigin_Function (line 444) | type isOrigin_Function interface
type Origin_PythonOrigin (line 448) | type Origin_PythonOrigin struct
method isOrigin_Function (line 456) | func (*Origin_PythonOrigin) isOrigin_Function() {}
type Origin_JvmOrigin (line 452) | type Origin_JvmOrigin struct
method isOrigin_Function (line 458) | func (*Origin_JvmOrigin) isOrigin_Function() {}
type PythonOrigin (line 460) | type PythonOrigin struct
method Reset (line 471) | func (x *PythonOrigin) Reset() {
method String (line 480) | func (x *PythonOrigin) String() string {
method ProtoMessage (line 484) | func (*PythonOrigin) ProtoMessage() {}
method ProtoReflect (line 486) | func (x *PythonOrigin) ProtoReflect() protoreflect.Message {
method Descriptor (line 499) | func (*PythonOrigin) Descriptor() ([]byte, []int) {
method GetFragment (line 503) | func (x *PythonOrigin) GetFragment() string {
method GetCallSite (line 510) | func (x *PythonOrigin) GetCallSite() string {
type JvmOrigin (line 517) | type JvmOrigin struct
method Reset (line 540) | func (x *JvmOrigin) Reset() {
method String (line 549) | func (x *JvmOrigin) String() string {
method ProtoMessage (line 553) | func (*JvmOrigin) ProtoMessage() {}
method ProtoReflect (line 555) | func (x *JvmOrigin) ProtoReflect() protoreflect.Message {
method Descriptor (line 568) | func (*JvmOrigin) Descriptor() ([]byte, []int) {
method GetLine (line 572) | func (x *JvmOrigin) GetLine() int32 {
method GetStartPosition (line 579) | func (x *JvmOrigin) GetStartPosition() int32 {
method GetStartIndex (line 586) | func (x *JvmOrigin) GetStartIndex() int32 {
method GetStopIndex (line 593) | func (x *JvmOrigin) GetStopIndex() int32 {
method GetSqlText (line 600) | func (x *JvmOrigin) GetSqlText() string {
method GetObjectType (line 607) | func (x *JvmOrigin) GetObjectType() string {
method GetObjectName (line 614) | func (x *JvmOrigin) GetObjectName() string {
method GetStackTrace (line 621) | func (x *JvmOrigin) GetStackTrace() []*StackTraceElement {
type StackTraceElement (line 629) | type StackTraceElement struct
method Reset (line 650) | func (x *StackTraceElement) Reset() {
method String (line 659) | func (x *StackTraceElement) String() string {
method ProtoMessage (line 663) | func (*StackTraceElement) ProtoMessage() {}
method ProtoReflect (line 665) | func (x *StackTraceElement) ProtoReflect() protoreflect.Message {
method Descriptor (line 678) | func (*StackTraceElement) Descriptor() ([]byte, []int) {
method GetClassLoaderName (line 682) | func (x *StackTraceElement) GetClassLoaderName() string {
method GetModuleName (line 689) | func (x *StackTraceElement) GetModuleName() string {
method GetModuleVersion (line 696) | func (x *StackTraceElement) GetModuleVersion() string {
method GetDeclaringClass (line 703) | func (x *StackTraceElement) GetDeclaringClass() string {
method GetMethodName (line 710) | func (x *StackTraceElement) GetMethodName() string {
method GetFileName (line 717) | func (x *StackTraceElement) GetFileName() string {
method GetLineNumber (line 724) | func (x *StackTraceElement) GetLineNumber() int32 {
type Bools (line 731) | type Bools struct
method Reset (line 739) | func (x *Bools) Reset() {
method String (line 748) | func (x *Bools) String() string {
method ProtoMessage (line 752) | func (*Bools) ProtoMessage() {}
method ProtoReflect (line 754) | func (x *Bools) ProtoReflect() protoreflect.Message {
method Descriptor (line 767) | func (*Bools) Descriptor() ([]byte, []int) {
method GetValues (line 771) | func (x *Bools) GetValues() []bool {
type Ints (line 778) | type Ints struct
method Reset (line 786) | func (x *Ints) Reset() {
method String (line 795) | func (x *Ints) String() string {
method ProtoMessage (line 799) | func (*Ints) ProtoMessage() {}
method ProtoReflect (line 801) | func (x *Ints) ProtoReflect() protoreflect.Message {
method Descriptor (line 814) | func (*Ints) Descriptor() ([]byte, []int) {
method GetValues (line 818) | func (x *Ints) GetValues() []int32 {
type Longs (line 825) | type Longs struct
method Reset (line 833) | func (x *Longs) Reset() {
method String (line 842) | func (x *Longs) String() string {
method ProtoMessage (line 846) | func (*Longs) ProtoMessage() {}
method ProtoReflect (line 848) | func (x *Longs) ProtoReflect() protoreflect.Message {
method Descriptor (line 861) | func (*Longs) Descriptor() ([]byte, []int) {
method GetValues (line 865) | func (x *Longs) GetValues() []int64 {
type Floats (line 872) | type Floats struct
method Reset (line 880) | func (x *Floats) Reset() {
method String (line 889) | func (x *Floats) String() string {
method ProtoMessage (line 893) | func (*Floats) ProtoMessage() {}
method ProtoReflect (line 895) | func (x *Floats) ProtoReflect() protoreflect.Message {
method Descriptor (line 908) | func (*Floats) Descriptor() ([]byte, []int) {
method GetValues (line 912) | func (x *Floats) GetValues() []float32 {
type Doubles (line 919) | type Doubles struct
method Reset (line 927) | func (x *Doubles) Reset() {
method String (line 936) | func (x *Doubles) String() string {
method ProtoMessage (line 940) | func (*Doubles) ProtoMessage() {}
method ProtoReflect (line 942) | func (x *Doubles) ProtoReflect() protoreflect.Message {
method Descriptor (line 955) | func (*Doubles) Descriptor() ([]byte, []int) {
method GetValues (line 959) | func (x *Doubles) GetValues() []float64 {
type Strings (line 966) | type Strings struct
method Reset (line 974) | func (x *Strings) Reset() {
method String (line 983) | func (x *Strings) String() string {
method ProtoMessage (line 987) | func (*Strings) ProtoMessage() {}
method ProtoReflect (line 989) | func (x *Strings) ProtoReflect() protoreflect.Message {
method Descriptor (line 1002) | func (*Strings) Descriptor() ([]byte, []int) {
method GetValues (line 1006) | func (x *Strings) GetValues() []string {
function file_spark_connect_common_proto_rawDescGZIP (line 1167) | func file_spark_connect_common_proto_rawDescGZIP() []byte {
function init (line 1209) | func init() { file_spark_connect_common_proto_init() }
function file_spark_connect_common_proto_init (line 1210) | func file_spark_connect_common_proto_init() {
FILE: internal/generated/example_plugins.pb.go
constant _ (line 34) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 36) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type ExamplePluginRelation (line 39) | type ExamplePluginRelation struct
method Reset (line 48) | func (x *ExamplePluginRelation) Reset() {
method String (line 57) | func (x *ExamplePluginRelation) String() string {
method ProtoMessage (line 61) | func (*ExamplePluginRelation) ProtoMessage() {}
method ProtoReflect (line 63) | func (x *ExamplePluginRelation) ProtoReflect() protoreflect.Message {
method Descriptor (line 76) | func (*ExamplePluginRelation) Descriptor() ([]byte, []int) {
method GetInput (line 80) | func (x *ExamplePluginRelation) GetInput() *Relation {
method GetCustomField (line 87) | func (x *ExamplePluginRelation) GetCustomField() string {
type ExamplePluginExpression (line 94) | type ExamplePluginExpression struct
method Reset (line 103) | func (x *ExamplePluginExpression) Reset() {
method String (line 112) | func (x *ExamplePluginExpression) String() string {
method ProtoMessage (line 116) | func (*ExamplePluginExpression) ProtoMessage() {}
method ProtoReflect (line 118) | func (x *ExamplePluginExpression) ProtoReflect() protoreflect.Message {
method Descriptor (line 131) | func (*ExamplePluginExpression) Descriptor() ([]byte, []int) {
method GetChild (line 135) | func (x *ExamplePluginExpression) GetChild() *Expression {
method GetCustomField (line 142) | func (x *ExamplePluginExpression) GetCustomField() string {
type ExamplePluginCommand (line 149) | type ExamplePluginCommand struct
method Reset (line 157) | func (x *ExamplePluginCommand) Reset() {
method String (line 166) | func (x *ExamplePluginCommand) String() string {
method ProtoMessage (line 170) | func (*ExamplePluginCommand) ProtoMessage() {}
method ProtoReflect (line 172) | func (x *ExamplePluginCommand) ProtoReflect() protoreflect.Message {
method Descriptor (line 185) | func (*ExamplePluginCommand) Descriptor() ([]byte, []int) {
method GetCustomField (line 189) | func (x *ExamplePluginCommand) GetCustomField() string {
function file_spark_connect_example_plugins_proto_rawDescGZIP (line 235) | func file_spark_connect_example_plugins_proto_rawDescGZIP() []byte {
function init (line 260) | func init() { file_spark_connect_example_plugins_proto_init() }
function file_spark_connect_example_plugins_proto_init (line 261) | func file_spark_connect_example_plugins_proto_init() {
FILE: internal/generated/expressions.pb.go
constant _ (line 35) | _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
constant _ (line 37) | _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
type Expression_Window_WindowFrame_FrameType (line 40) | type Expression_Window_WindowFrame_FrameType
method Enum (line 65) | func (x
Condensed preview — 104 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (2,295K chars).
[
{
"path": ".asf.yaml",
"chars": 1274,
"preview": "# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE fi"
},
{
"path": ".github/PULL_REQUEST_TEMPLATE",
"chars": 3092,
"preview": "<!--\nThanks for sending a pull request! Here are some tips for you:\n 1. If this is your first time, please read our co"
},
{
"path": ".github/dependabot.yml",
"chars": 527,
"preview": "# To get started with Dependabot version updates, you'll need to specify which\n# package ecosystems to update and where "
},
{
"path": ".github/workflows/build.yml",
"chars": 4825,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one\n# or more contributor license agreements. See the NOTICE"
},
{
"path": ".gitignore",
"chars": 1106,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE"
},
{
"path": ".gitmodules",
"chars": 124,
"preview": "[submodule \"spark\"]\n\tpath = sparksrc\n\turl = https://github.com/apache/spark.git\n[submodule \"sparksrc\"]\n\tbranch = branch-"
},
{
"path": ".golangci.yml",
"chars": 1145,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE"
},
{
"path": "CONTRIBUTING.md",
"chars": 3321,
"preview": "## Contributing to Spark\n\n*Before opening a pull request*, review the\n[Contributing to Spark guide](https://spark.apache"
},
{
"path": "LICENSE",
"chars": 11357,
"preview": " Apache License\n Version 2.0, January 2004\n "
},
{
"path": "Makefile",
"chars": 4494,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE"
},
{
"path": "NOTICE",
"chars": 173,
"preview": "Apache Spark\nCopyright 2014 and onwards The Apache Software Foundation.\n\nThis product includes software developed at\nThe"
},
{
"path": "README.md",
"chars": 2576,
"preview": "# Apache Spark Connect Client for Golang\n\nThis project houses the **experimental** client for [Spark\nConnect](https://sp"
},
{
"path": "buf.gen.yaml",
"chars": 914,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE"
},
{
"path": "buf.work.yaml",
"chars": 859,
"preview": "#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE"
},
{
"path": "cmd/spark-connect-example-raw-grpc-client/main.go",
"chars": 1892,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "cmd/spark-connect-example-spark-session/main.go",
"chars": 5651,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "dev/.rat-excludes",
"chars": 205,
"preview": ".gitignore\n.gitmodules\n.gitattributes\n.project\ncoverage*\nLICENSE\nNOTICE\nTAGS\nRELEASE\n.*md\n.rat-excludes\nsparksrc\ntarget\n"
},
{
"path": "dev/README.md",
"chars": 3528,
"preview": "# Release Script for Apache Spark Connect Go\n\nThis directory contains the release automation script for the Apache Spark"
},
{
"path": "dev/check-license",
"chars": 2561,
"preview": "#!/usr/bin/env bash\n\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreem"
},
{
"path": "dev/gen.py",
"chars": 6221,
"preview": "# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. See the NOTICE f"
},
{
"path": "dev/release.py",
"chars": 10727,
"preview": "#!/usr/bin/env python3\n\"\"\"\nLicensed to the Apache Software Foundation (ASF) under one or more\ncontributor license agreem"
},
{
"path": "dev/requirements.txt",
"chars": 51,
"preview": "requests>=2.28.0\nPyGithub>=1.58.0\ngitpython>=3.1.30"
},
{
"path": "go.mod",
"chars": 2232,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "go.sum",
"chars": 9285,
"preview": "cloud.google.com/go/compute/metadata v0.7.0 h1:PBWF+iiAerVNe8UCHxdOt6eHLVc3ydFeOCw78U8ytSU=\ncloud.google.com/go/compute/"
},
{
"path": "internal/generated/base.pb.go",
"chars": 363750,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/base_grpc.pb.go",
"chars": 25001,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/catalog.pb.go",
"chars": 90897,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/commands.pb.go",
"chars": 228923,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/common.pb.go",
"chars": 50005,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/example_plugins.pb.go",
"chars": 12160,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/expressions.pb.go",
"chars": 192435,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/ml.pb.go",
"chars": 45431,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/ml_common.pb.go",
"chars": 15116,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/pipelines.pb.go",
"chars": 59425,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/relations.pb.go",
"chars": 342684,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/generated/types.pb.go",
"chars": 93035,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "internal/tests/integration/dataframe_test.go",
"chars": 35857,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "internal/tests/integration/functions_test.go",
"chars": 2478,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "internal/tests/integration/helper.go",
"chars": 1947,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "internal/tests/integration/spark_runner.go",
"chars": 2245,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "internal/tests/integration/sql_test.go",
"chars": 3802,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "java/.gitignore",
"chars": 14,
"preview": "project\ntarget"
},
{
"path": "java/README.md",
"chars": 1103,
"preview": "# Sample Spark-Submit Wrapper\n\nThis directory provides a simple wrapper library that can be used to submit a Spark Conne"
},
{
"path": "java/build.sbt",
"chars": 1324,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "java/run.sh",
"chars": 1628,
"preview": "#!/bin/bash\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agreements. Se"
},
{
"path": "java/src/main/scala/org/apache/spark/golang/Runner.scala",
"chars": 2132,
"preview": "/*\n * Licensed to the Apache Software Foundation (ASF) under one or more\n * contributor license agreements. See the NOT"
},
{
"path": "merge_connect_go_pr.py",
"chars": 26149,
"preview": "#!/usr/bin/env python3\n\n#\n# Licensed to the Apache Software Foundation (ASF) under one or more\n# contributor license agr"
},
{
"path": "quick-start.md",
"chars": 3070,
"preview": "# Quick Start Guide to Write Spark Connect Client Application in Go\n\n## Add Reference to `spark-connect-go` Library\n\nIn "
},
{
"path": "spark/client/base/base.go",
"chars": 2565,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/channel/channel.go",
"chars": 7162,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/client/channel/channel_test.go",
"chars": 4265,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/client/channel/compat.go",
"chars": 967,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/client.go",
"chars": 15887,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/client_test.go",
"chars": 4311,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/conf.go",
"chars": 4297,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/options/options.go",
"chars": 1130,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/retry.go",
"chars": 16701,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/retry_test.go",
"chars": 9560,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/client/testutils/utils.go",
"chars": 3600,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/mocks/mock_executor.go",
"chars": 3174,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/mocks/mocks.go",
"chars": 4821,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sparkerrors/errors.go",
"chars": 6051,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sparkerrors/errors_test.go",
"chars": 5704,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/column/column.go",
"chars": 4007,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/column/column_test.go",
"chars": 9818,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/column/expressions.go",
"chars": 9383,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/column/expressions_test.go",
"chars": 5256,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/dataframe.go",
"chars": 55791,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/dataframe_test.go",
"chars": 1549,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/dataframenafunctions.go",
"chars": 2658,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/dataframereader.go",
"chars": 2590,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/dataframereader_test.go",
"chars": 1925,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/dataframestatfunctions.go",
"chars": 3776,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/dataframewriter.go",
"chars": 3462,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/dataframewriter_test.go",
"chars": 3305,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/executeplanclient.go",
"chars": 810,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/functions/buiitins.go",
"chars": 2021,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/functions/generated.go",
"chars": 166176,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/group.go",
"chars": 6219,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/group_test.go",
"chars": 3950,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/mocks_test.go",
"chars": 810,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/plan.go",
"chars": 2087,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/plan_test.go",
"chars": 991,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/sparksession.go",
"chars": 8737,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/sparksession_integration_test.go",
"chars": 2884,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/sparksession_test.go",
"chars": 5381,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/arrow.go",
"chars": 10375,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/arrow_test.go",
"chars": 13648,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/builtin.go",
"chars": 9545,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/builtin_test.go",
"chars": 2557,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/conversion.go",
"chars": 3513,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/conversion_test.go",
"chars": 8479,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/datatype.go",
"chars": 6013,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/datatype_test.go",
"chars": 5551,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/row.go",
"chars": 6580,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/row_json_test.go",
"chars": 5351,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/row_test.go",
"chars": 2396,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/structtype.go",
"chars": 3160,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/types/structtype_test.go",
"chars": 6203,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/utils/check.go",
"chars": 908,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/utils/check_test.go",
"chars": 1244,
"preview": "//\n// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOT"
},
{
"path": "spark/sql/utils/consts.go",
"chars": 5228,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/sql/utils/consts_test.go",
"chars": 1484,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
},
{
"path": "spark/version.go",
"chars": 852,
"preview": "// Licensed to the Apache Software Foundation (ASF) under one or more\n// contributor license agreements. See the NOTICE"
}
]
About this extraction
This page contains the full source code of the apache/spark-connect-go GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 104 files (2.1 MB), approximately 546.1k tokens, and a symbol index with 5233 extracted functions, classes, methods, constants, and types. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.
Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.