diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 068c0d1b56fd..5aae8146e462 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -56,6 +56,7 @@ dependencies { implementation project(":sdks:java:io:google-cloud-platform") implementation project(":sdks:java:io:kafka") implementation project(":sdks:java:extensions:ml") + implementation project(":sdks:java:managed") implementation library.java.avro implementation library.java.bigdataoss_util implementation library.java.google_api_client diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/IcebergBeamSchemaAndRow.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/IcebergBeamSchemaAndRow.java new file mode 100644 index 000000000000..bdbd59ebd8d7 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/IcebergBeamSchemaAndRow.java @@ -0,0 +1,80 @@ +/* + * 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 org.apache.beam.examples.snippets.transforms.io.iceberg; + +// [START iceberg_schema_and_row] + +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.Arrays; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; +import org.apache.beam.sdk.schemas.logicaltypes.Timestamp; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.DateTime; + +public class IcebergBeamSchemaAndRow { + Schema nestedSchema = + Schema.builder().addStringField("nested_field").addInt32Field("nested_field_2").build(); + Schema beamSchema = + Schema.builder() + .addBooleanField("boolean_field") + .addInt32Field("int_field") + .addInt64Field("long_field") + .addFloatField("float_field") + .addDoubleField("double_field") + .addDecimalField("numeric_field") + .addByteArrayField("bytes_field") + .addStringField("string_field") + .addLogicalTypeField("time_field", SqlTypes.TIME) + .addLogicalTypeField("date_field", SqlTypes.DATE) + .addLogicalTypeField("timestamp_field", Timestamp.MICROS) + .addDateTimeField("timestamptz_field") + .addArrayField("array_field", Schema.FieldType.INT32) + .addMapField("map_field", Schema.FieldType.STRING, Schema.FieldType.INT32) + .addRowField("struct_field", nestedSchema) + .build(); + + Row beamRow = + Row.withSchema(beamSchema) + .withFieldValues( + ImmutableMap.builder() + .put("boolean_field", true) + .put("int_field", 1) + .put("long_field", 2L) + .put("float_field", 3.4f) + .put("double_field", 4.5d) + .put("numeric_field", new BigDecimal(67)) + .put("bytes_field", new byte[] {1, 2, 3}) + .put("string_field", "value") + .put("time_field", LocalTime.now()) + .put("date_field", LocalDate.now()) + .put("timestamp_field", Instant.now()) + .put("timestamptz_field", DateTime.now()) + .put("array_field", Arrays.asList(1, 2, 3)) + .put("map_field", ImmutableMap.of("a", 1, "b", 2)) + .put( + "struct_field", + Row.withSchema(nestedSchema).addValues("nested_value", 123).build()) + .build()) + .build(); +} +// [END iceberg_schema_and_row] diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java new file mode 100644 index 000000000000..fa519f748f6c --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java @@ -0,0 +1,91 @@ +/* + * 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 org.apache.beam.examples.snippets.transforms.io.iceberg; + +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +public class Quickstart { + public static void main(String[] args) { + // [START hadoop_catalog_props] + Map catalogProps = + ImmutableMap.of( + "type", "hadoop", + "warehouse", "file://tmp/beam-iceberg-local-quickstart"); + // [END hadoop_catalog_props] + } + + public static void other() { + // [START biglake_catalog_props] + Map catalogProps = + ImmutableMap.of( + "type", "rest", + "uri", "https://biglake.googleapis.com/iceberg/v1/restcatalog", + "warehouse", "gs://biglake-public-nyc-taxi-iceberg", + "header.x-goog-user-project", "$PROJECT_ID", + "rest.auth.type", "google", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "header.X-Iceberg-Access-Delegation", "vended-credentials"); + // [END biglake_catalog_props] + + // [START managed_iceberg_config] + Map managedConfig = + ImmutableMap.of("table", "my_db.my_table", "catalog_properties", catalogProps); + // Note: The table will get created when inserting data (see below) + // [END managed_iceberg_config] + + // [START managed_iceberg_insert] + Schema inputSchema = + Schema.builder().addInt64Field("id").addStringField("name").addInt32Field("age").build(); + + Pipeline p = Pipeline.create(); + p.apply( + Create.of( + Row.withSchema(inputSchema).addValues(1, "Mark", 34).build(), + Row.withSchema(inputSchema).addValues(2, "Omar", 24).build(), + Row.withSchema(inputSchema).addValues(3, "Rachel", 27).build())) + .apply(Managed.write("iceberg").withConfig(managedConfig)); + + p.run(); + // [END managed_iceberg_insert] + + // [START managed_iceberg_read] + Pipeline q = Pipeline.create(); + PCollection rows = + q.apply(Managed.read("iceberg").withConfig(managedConfig)).getSinglePCollection(); + + rows.apply( + MapElements.into(TypeDescriptors.voids()) + .via( + row -> { + System.out.println(row); + return null; + })); + + q.run(); + // [END managed_iceberg_read] + } +} diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index f6bf5e5d44ec..8edcaace70d9 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -1077,6 +1077,74 @@ def model_bigqueryio_xlang( # [END model_bigqueryio_write_with_storage_write_api] +def model_managed_iceberg(): + """Examples for Managed Iceberg sources and sinks.""" + # [START hadoop_catalog_config] + hadoop_catalog_props = { + 'type': 'hadoop', 'warehouse': 'file://tmp/beam-iceberg-local-quickstart' + } + # [END hadoop_catalog_config] + + # [START managed_iceberg_config] + managed_config = { + 'table': 'my_db.my_table', 'catalog_properties': hadoop_catalog_props + } + # Note: The table will get created when inserting data (see below) + # [END managed_iceberg_config] + + # [START managed_iceberg_insert] + with beam.Pipeline() as p: + ( + p + | beam.Create([ + beam.Row(id=1, name="Mark", age=32), + beam.Row(id=2, name="Omar", age=24), + beam.Row(id=3, name="Rachel", age=27) + ]) + | beam.managed.Write("iceberg", config=managed_config)) + # [END managed_iceberg_insert] + + # [START managed_iceberg_read] + with beam.Pipeline() as p: + ( + p + | beam.managed.Read("iceberg", config=managed_config) + | beam.LogElements()) + # [END managed_iceberg_read] + + # [START biglake_catalog_config] + biglake_catalog_config = { + 'type': 'rest', + 'uri': 'https://biglake.googleapis.com/iceberg/v1/restcatalog', + 'warehouse': 'gs://biglake-public-nyc-taxi-iceberg', + 'header.x-goog-user-project': '$PROJECT_ID', + 'rest.auth.type': 'google', + 'io-impl': 'org.apache.iceberg.gcp.gcs.GCSFileIO', + 'header.X-Iceberg-Access-Delegation': 'vended-credentials' + } + # [END biglake_catalog_config] + + # [START model_managed_iceberg_data_types] + from decimal import Decimal + import apache_beam as beam + from apache_beam.utils.timestamp import Timestamp + + row = beam.Row( + boolean_field=True, + int_field=1, + float_field=2.3, + numeric_field=Decimal('34'), + bytes_field=b'value', + string_field="value", + timestamptz_field=Timestamp(4, 5), + array_field=[1, 2, 3], + map_field={ + "a": 1, "b": 2 + }, + struct_field=beam.Row(nested_field="nested_value", nested_field2=123)) + # [END model_managed_iceberg_data_types] + + def model_composite_transform_example(contents, output_path): """Example of a composite transform. diff --git a/website/www/site/assets/js/language-switch-v2.js b/website/www/site/assets/js/language-switch-v2.js index f4dd73a5b04f..ff6fdb0253a8 100644 --- a/website/www/site/assets/js/language-switch-v2.js +++ b/website/www/site/assets/js/language-switch-v2.js @@ -61,6 +61,7 @@ $(document).ready(function() { "valueToTabTitle": function (value) { switch (value) { case 'py': return 'Python'; + case 'sql': return 'SQL'; case 'scio': return 'SCIO'; case 'typescript': return 'TypeScript'; } @@ -287,6 +288,7 @@ $(document).ready(function() { }).render(); Switcher({"name": "runner", "default": "direct"}).render(); + Switcher({"name": "tab"}).render(); Switcher({"name": "shell", "default": "unix"}).render(); Switcher({"name": "version"}).render(); }); diff --git a/website/www/site/assets/scss/_syntax-highlighting.scss b/website/www/site/assets/scss/_syntax-highlighting.scss index 8a347d3f74f8..fd9f3216349e 100644 --- a/website/www/site/assets/scss/_syntax-highlighting.scss +++ b/website/www/site/assets/scss/_syntax-highlighting.scss @@ -322,6 +322,9 @@ pre { } } } +.tab-switcher { + @extend .runner-switcher; +} .shell-switcher { ul.nav-tabs { padding-left: 0; diff --git a/website/www/site/content/en/documentation/io/built-in/iceberg.md b/website/www/site/content/en/documentation/io/built-in/iceberg.md new file mode 100644 index 000000000000..0c93cd57f99b --- /dev/null +++ b/website/www/site/content/en/documentation/io/built-in/iceberg.md @@ -0,0 +1,336 @@ +--- +title: "Apache Iceberg" +--- + + +[Built-in I/O Transforms](/documentation/io/built-in/) + + +# Apache Iceberg I/O connector + +The Beam SDKs include built-in transforms that can read data from and write data +to [Apache Iceberg](https://iceberg.apache.org/) tables. + +{{< language-switcher sql java py yaml>}} + +{{< paragraph class="language-java" >}} +To use IcebergIO, add the Maven artifact dependency to your `pom.xml` file. +{{< /paragraph >}} + +{{< highlight java >}} + + org.apache.beam + beam-sdks-java-io-iceberg + {{< param release_latest >}} + +{{< /highlight >}} + +{{< paragraph class="language-sql" >}} +To use IcebergIO, install the [Beam SQL Shell](https://beam.apache.org/documentation/dsls/sql/shell/#installation) and run the following command: +{{< /paragraph >}} + +{{% section class="language-sql" %}} +```shell +./beam-sql.sh --io iceberg +``` +{{% /section %}} + +{{< paragraph >}} +Additional resources: +{{< /paragraph >}} + +{{< paragraph wrap="span" >}} +* [IcebergIO configuration parameters](https://beam.apache.org/documentation/io/managed-io/#iceberg-write) +* [IcebergIO source code](https://github.com/apache/beam/tree/master/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg) +* [IcebergIO Javadoc](https://beam.apache.org/releases/javadoc/{{< param release_latest >}}/org/apache/beam/sdk/io/iceberg/IcebergIO.html) +* [Apache Iceberg spec](https://iceberg.apache.org/spec/) +* [Apache Iceberg terms](https://iceberg.apache.org/terms/) +{{< /paragraph >}} + +## Iceberg basics + +### Catalogs + +A catalog is a top-level entity used to manage and access Iceberg tables. There are many catalog implementations out there; +this guide focuses on the Hadoop catalog for easy local testing and BigLake REST catalog for cloud-scale development. + +### Namespaces + +A namespace lives inside a catalog and may contain a number of Iceberg tables. This is the equivalent of a "database". + +### Tables + +The actual entity containing data, and is described by a schema and partition spec. + +### Snapshots + +A new snapshot is created whenever a change is made to an Iceberg table. Each snapshot provides a summary of the change +and references its parent snapshot. An Iceberg table's history is a chronological list of snapshots, enabling features +like time travel and ACID-compliant concurrent writes. + +## Quickstart Guide + +### Choose Your Catalog + +First, select a Catalog implementation to handle metadata management and storage interaction. +Beam supports a wide variety of Iceberg catalogs, but this guide focuses on two common paths: +**Hadoop** for easy local development and **BigLake** for managing production data at cloud scale. + +{{< tab hadoop >}} +

+ Use Hadoop Catalog for quick, local testing with zero setup and no external dependencies. + The following examples use a temporary local directory. +

+ +
+ {{< highlight sql >}} + CREATE CATALOG my_catalog TYPE 'iceberg' + PROPERTIES ( + 'type' = 'hadoop', + 'warehouse' = 'file://tmp/beam-iceberg-local-quickstart', + ); + {{< /highlight >}} + {{< highlight java>}} + {{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java" hadoop_catalog_props >}} + {{< /highlight >}} + {{< highlight py >}} + {{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" hadoop_catalog_config >}} + {{< /highlight >}} + {{< highlight yaml >}} + catalog_props: &catalog_props + type: "hadoop" + warehouse: "file://tmp/beam-iceberg-local-quickstart" + {{< /highlight >}} +{{< /tab >}} +{{< tab BigLake >}} +{{% section %}} +Use BigLake Catalog for a fully managed REST-based experience. It simplifies access to cloud storage with +built-in credential delegation and unified metadata management. It requires a few pre-requisites: + +- A Google Cloud Project (for authentication). Create an account [here](https://docs.cloud.google.com/docs/get-started) if you don't have one. +- Standard Google [Application Default Credentials](https://docs.cloud.google.com/docs/authentication/set-up-adc-local-dev-environment#local-user-cred) (ADC) set up in your environment. +- A [Google Cloud Storage bucket](https://docs.cloud.google.com/storage/docs/creating-buckets) + +{{% /section %}} + {{< highlight sql>}} + CREATE CATALOG my_catalog TYPE 'iceberg' + PROPERTIES ( + 'type' = 'rest', + 'uri' = 'https://biglake.googleapis.com/iceberg/v1/restcatalog', + 'warehouse' = 'gs://$BUCKET_NAME', + 'header.x-goog-user-project' = '$PROJECT_ID', + 'rest.auth.type' = 'google', + 'io-impl' = 'org.apache.iceberg.gcp.gcs.GCSFileIO', + 'header.X-Iceberg-Access-Delegation' = 'vended-credentials' + ); + {{< /highlight >}} + {{< highlight java>}} + {{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java" biglake_catalog_props >}} + {{< /highlight >}} + {{< highlight py >}} + {{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" biglake_catalog_config >}} + {{< /highlight >}} + {{< highlight yaml >}} + catalog_props: &catalog_props + type: "rest" + uri: "https://biglake.googleapis.com/iceberg/v1/restcatalog" + warehouse: "gs://$BUCKET_NAME" + header.x-goog-user-project: "$PROJECT_ID" + rest.auth.type: "google" + io-impl: "org.apache.iceberg.gcp.gcs.GCSFileIO" + header.X-Iceberg-Access-Delegation: "vended-credentials" + {{< /highlight >}} +{{< /tab >}} + +### Create a Namespace + +You can use Beam SQL to create a new namespace through an explicit DDL statement: +```sql +CREATE DATABASE my_catalog.my_db; +``` + +Alternatively, the IcebergIO sink can handle namespace creation automatically at runtime. +This is ideal for dynamic pipelines where destinations are determined by the incoming data + +### Create a Table +Tables are defined by a schema and an optional partition spec. +You can create a table using SQL DDL or by configuring the Iceberg destination in your Beam pipeline. + +{{< highlight sql>}} +CREATE EXTERNAL TABLE my_catalog.my_db.my_table ( + id BIGINT, + name VARCHAR, + age INTEGER +) +TYPE 'iceberg' +{{< /highlight >}} +{{< highlight java>}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java" managed_iceberg_config >}} +{{< /highlight >}} +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" managed_iceberg_config >}} +{{< /highlight >}} +{{< highlight yaml >}} +- type: WriteToIceberg + config: + table: "my_db.my_table" + catalog_properties: *catalog_props + +# Note: The table will get created when inserting data (see below) +{{< /highlight >}} + +### Insert Data +Once your table is defined, you can write data using standard SQL `INSERT` or by calling the IcebergIO sink in your SDK of choice. + + +{{< highlight sql>}} +INSERT INTO my_catalog.my_db.my_table VALUES + (1, 'Mark', 32), + (2, 'Omar', 24), + (3, 'Rachel', 27); +{{< /highlight >}} +{{< highlight java>}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java" managed_iceberg_insert >}} +{{< /highlight >}} +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" managed_iceberg_insert >}} +{{< /highlight >}} +{{< highlight yaml >}} +pipeline: + type: chain + transforms: + - type: Create + config: + elements: + - id: 1 + name: "Mark" + age: 32 + - id: 2 + name: "Omar" + age: 24 + - id: 3 + name: "Rachel" + age: 27 + - type: WriteToIceberg + config: + table: "my_db.my_table" + catalog_properties: *catalog_props +{{< /highlight >}} + +### View Namespaces and Tables + +You can use Beam SQL to view the newly created resources: +```sql +SHOW DATABASES my_catalog; +``` +```sql +SHOW TABLES my_catalog.my_db; +``` + +### Query Data + +{{< highlight sql>}} +SELECT * FROM my_catalog.my_db.my_table; +{{< /highlight >}} +{{< highlight java>}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/Quickstart.java" managed_iceberg_read >}} +{{< /highlight >}} +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" managed_iceberg_read >}} +{{< /highlight >}} +{{< highlight yaml >}} +pipeline: + type: chain + transforms: + - type: ReadFromIceberg + config: + table: "my_db.my_table" + catalog_properties: *catalog_props + - type: LogForTesting +{{< /highlight >}} + + + +## Data Types + +Check this [overview of Iceberg data types](https://iceberg.apache.org/spec/#schemas-and-data-types). + +IcebergIO leverages Beam Schemas to bridge the gap between SDK-native types and the Iceberg specification. +While the Java SDK provides full coverage for the Iceberg v2 spec (with v3 support currently in development), +other SDKs may have specific constraints on complex or experimental types. The following examples demonstrate +the standard mapping for core data types across SQL, Java, Python, and YAML: + +{{< highlight sql >}} +INSERT INTO catalog.namespace.table VALUES ( +9223372036854775807, -- BIGINT +2147483647, -- INTEGER +1.0, -- FLOAT +1.0, -- DOUBLE +TRUE, -- BOOLEAN +TIMESTAMP '2018-05-28 20:17:40.123', -- TIMESTAMP +'varchar', -- VARCHAR +'char', -- CHAR +ARRAY['abc', 'xyz'], -- ARRAY +ARRAY[CAST(ROW('abc', 123) AS ROW(nested_str VARCHAR, nested_int INTEGER))] -- ARRAY[STRUCT] +) +{{< /highlight >}} +{{< highlight java >}} +{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/iceberg/IcebergBeamSchemaAndRow.java" iceberg_schema_and_row >}} +{{< /highlight >}} +{{< highlight py >}} +{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_managed_iceberg_data_types >}} +{{< /highlight >}} +{{< highlight yaml >}} +pipeline: + transforms: + - type: Create + config: + elements: + - boolean_field: false + integer_field: 123 + number_field: 4.56 + string_field: "abc" + struct_field: + nested_1: a + nested_2: 1 + array_field: [1, 2, 3] + output_schema: + type: object + properties: + boolean_field: + type: boolean + integer_field: + type: integer + number_field: + type: number + string_field: + type: string + struct_field: + type: object + properties: + nested_1: + type: string + nested_2: + type: integer + array_field: + type: array + items: + type: integer +{{< /highlight >}} + +## Further steps + +Check out the full [IcebergIO configuration](https://beam.apache.org/documentation/io/managed-io/#iceberg-write) to make +use of other features like applying a partition spec, table properties, row filtering, column pruning, etc. diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html index 0cc197d95fdc..97b694538ee8 100755 --- a/website/www/site/layouts/partials/section-menu/en/documentation.html +++ b/website/www/site/layouts/partials/section-menu/en/documentation.html @@ -69,15 +69,16 @@
  • I/O connector guides
  • diff --git a/website/www/site/layouts/shortcodes/language-switcher.html b/website/www/site/layouts/shortcodes/language-switcher.html index cc9b5864a3b8..2f50691e3cca 100644 --- a/website/www/site/layouts/shortcodes/language-switcher.html +++ b/website/www/site/layouts/shortcodes/language-switcher.html @@ -20,6 +20,9 @@ {{ if eq $lang "py" }}
  • Python SDK
  • {{ end }} + {{ if eq $lang "sql" }} +
  • SQL Shell
  • + {{ end }} {{ if eq $lang "go" }}
  • Go SDK
  • {{ end }} diff --git a/website/www/site/layouts/shortcodes/section.html b/website/www/site/layouts/shortcodes/section.html new file mode 100644 index 000000000000..5b7f14fe0f05 --- /dev/null +++ b/website/www/site/layouts/shortcodes/section.html @@ -0,0 +1,15 @@ +{{/* +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. See accompanying LICENSE file. +*/}} + +
    +{{ .Inner | markdownify }} +
    diff --git a/website/www/site/layouts/shortcodes/tab.html b/website/www/site/layouts/shortcodes/tab.html new file mode 100644 index 000000000000..a5d6ecd607a4 --- /dev/null +++ b/website/www/site/layouts/shortcodes/tab.html @@ -0,0 +1,27 @@ +{{/* +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. See accompanying LICENSE file. +*/}} + +{{ $content := (trim .Inner "\n\r") | htmlUnescape | safeHTML }} +{{ $ctx := . }} +{{ $language := .Get 0 }} +{{ with $language }} +
    + {{ $content }} +
    +{{ end }}