Skip to content

Commit

Permalink
add json type
Browse files Browse the repository at this point in the history
  • Loading branch information
SpencerTorres committed Dec 20, 2024
1 parent 33abd04 commit b37eca7
Show file tree
Hide file tree
Showing 13 changed files with 2,720 additions and 781 deletions.
6 changes: 6 additions & 0 deletions chcol.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ type (
VariantWithType = chcol.VariantWithType
Dynamic = chcol.Dynamic
DynamicWithType = chcol.DynamicWithType
JSON = chcol.JSON
)

// NewVariant creates a new Variant with the given value
Expand All @@ -47,3 +48,8 @@ func NewDynamic(v any) Dynamic {
func NewDynamicWithType(v any, chType string) DynamicWithType {
return chcol.NewDynamicWithType(v, chType)
}

// NewJSON creates a new empty JSON value
func NewJSON() *JSON {
return chcol.NewJSON()
}
81 changes: 81 additions & 0 deletions examples/clickhouse_api/json_paths.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
// Licensed to ClickHouse, Inc. under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. ClickHouse, Inc. 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 clickhouse_api

import (
"context"
"fmt"
"github.com/ClickHouse/clickhouse-go/v2"
"time"
)

func JSONPathsExample() error {
ctx := context.Background()

conn, err := GetNativeConnection(clickhouse.Settings{
"allow_experimental_json_type": true,
}, nil, nil)
if err != nil {
return err
}

err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_json_example")
if err != nil {
return err
}

err = conn.Exec(ctx, `
CREATE TABLE go_json_example (product JSON) ENGINE=Memory
`)
if err != nil {
return err
}

batch, err := conn.PrepareBatch(ctx, "INSERT INTO go_json_example (product)")
if err != nil {
return err
}

insertProduct := clickhouse.NewJSON()
insertProduct.SetValueAtPath("id", clickhouse.NewDynamicWithType(uint64(1234), "UInt64"))
insertProduct.SetValueAtPath("name", "Book")
insertProduct.SetValueAtPath("tags", []string{"library", "fiction"})
insertProduct.SetValueAtPath("pricing.price", int64(750))
insertProduct.SetValueAtPath("pricing.currency", "usd")
insertProduct.SetValueAtPath("metadata.region", "us")
insertProduct.SetValueAtPath("metadata.page_count", int64(852))
insertProduct.SetValueAtPath("created_at", clickhouse.NewDynamicWithType(time.Now().UTC().Truncate(time.Millisecond), "DateTime64(3)"))

if err = batch.Append(insertProduct); err != nil {
return err
}

if err = batch.Send(); err != nil {
return err
}

var selectedProduct clickhouse.JSON

if err = conn.QueryRow(ctx, "SELECT product FROM go_json_example").Scan(&selectedProduct); err != nil {
return err
}

fmt.Printf("inserted product: %+v\n", insertProduct)
fmt.Printf("selected product: %+v\n", selectedProduct)
return nil
}
76 changes: 76 additions & 0 deletions examples/clickhouse_api/json_strings.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
// Licensed to ClickHouse, Inc. under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. ClickHouse, Inc. 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 clickhouse_api

import (
"context"
"fmt"
"github.com/ClickHouse/clickhouse-go/v2"
)

func JSONStringExample() error {
ctx := context.Background()

conn, err := GetNativeConnection(clickhouse.Settings{
"allow_experimental_json_type": true,
"output_format_native_write_json_as_string": true,
}, nil, nil)
if err != nil {
return err
}

err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_json_example")
if err != nil {
return err
}

err = conn.Exec(ctx, `
CREATE TABLE go_json_example (product JSON) ENGINE=Memory
`)
if err != nil {
return err
}

batch, err := conn.PrepareBatch(ctx, "INSERT INTO go_json_example (product)")
if err != nil {
return err
}

insertProductString := "{\"id\":1234,\"name\":\"Book\",\"tags\":[\"library\",\"fiction\"]," +
"\"pricing\":{\"price\":750,\"currency\":\"usd\"},\"metadata\":{\"page_count\":852,\"region\":\"us\"}," +
"\"created_at\":\"2024-12-19T11:20:04.146Z\"}"

if err = batch.Append(insertProductString); err != nil {
return err
}

if err = batch.Send(); err != nil {
return err
}

var selectedProductString string

if err = conn.QueryRow(ctx, "SELECT product FROM go_json_example").Scan(&selectedProductString); err != nil {
return err
}

fmt.Printf("inserted product string: %s\n", insertProductString)
fmt.Printf("selected product string: %s\n", selectedProductString)
fmt.Printf("inserted product string matches selected product string: %t\n", insertProductString == selectedProductString)
return nil
}
116 changes: 116 additions & 0 deletions examples/clickhouse_api/json_structs.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
// Licensed to ClickHouse, Inc. under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. ClickHouse, Inc. 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 clickhouse_api

import (
"context"
"encoding/json"
"fmt"
"github.com/ClickHouse/clickhouse-go/v2"
"time"
)

type ProductPricing struct {
Price int64 `json:"price"`
Currency string `json:"currency"`
}

type Product struct {
ID clickhouse.DynamicWithType `json:"id"`
Name string `json:"name"`
Tags []string `json:"tags"`
Pricing ProductPricing `json:"pricing"`
Metadata map[string]interface{} `json:"metadata"`
CreatedAt time.Time `json:"created_at" chType:"DateTime64(3)"`
}

func NewExampleProduct() *Product {
return &Product{
ID: clickhouse.NewDynamicWithType(uint64(1234), "UInt64"),
Name: "Book",
Tags: []string{"library", "fiction"},
Pricing: ProductPricing{
Price: 750,
Currency: "usd",
},
Metadata: map[string]interface{}{
"region": "us",
"page_count": int64(852),
},
CreatedAt: time.Now().UTC().Truncate(time.Millisecond),
}
}

func JSONStructExample() error {
ctx := context.Background()

conn, err := GetNativeConnection(clickhouse.Settings{
"allow_experimental_json_type": true,
}, nil, nil)
if err != nil {
return err
}

err = conn.Exec(ctx, "DROP TABLE IF EXISTS go_json_example")
if err != nil {
return err
}

err = conn.Exec(ctx, `
CREATE TABLE go_json_example (product JSON) ENGINE=Memory
`)
if err != nil {
return err
}

batch, err := conn.PrepareBatch(ctx, "INSERT INTO go_json_example (product)")
if err != nil {
return err
}

insertProduct := NewExampleProduct()

if err = batch.Append(insertProduct); err != nil {
return err
}

if err = batch.Send(); err != nil {
return err
}

var selectedProduct Product

if err = conn.QueryRow(ctx, "SELECT product FROM go_json_example").Scan(&selectedProduct); err != nil {
return err
}

insertProductBytes, err := json.Marshal(insertProduct)
if err != nil {
return err
}

selectedProductBytes, err := json.Marshal(&selectedProduct)
if err != nil {
return err
}

fmt.Printf("inserted product: %s\n", string(insertProductBytes))
fmt.Printf("selected product: %s\n", string(selectedProductBytes))
fmt.Printf("inserted product matches selected product: %t\n", string(insertProductBytes) == string(selectedProductBytes))
return nil
}
13 changes: 13 additions & 0 deletions examples/clickhouse_api/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -217,3 +217,16 @@ func TestVariantExample(t *testing.T) {
func TestDynamicExample(t *testing.T) {
require.NoError(t, DynamicExample())
}

func TestJSONPathsExample(t *testing.T) {
require.NoError(t, JSONPathsExample())
}

func TestJSONStructExample(t *testing.T) {
require.NoError(t, JSONStructExample())
}

func TestJSONStringExample(t *testing.T) {
t.Skip("client cannot receive JSON strings")
require.NoError(t, JSONStringExample())
}
70 changes: 70 additions & 0 deletions lib/chcol/json.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
// Licensed to ClickHouse, Inc. under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. ClickHouse, Inc. 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 chcol

import (
"database/sql/driver"
"encoding/json"
"fmt"
)

// JSON represents a ClickHouse JSON type that can hold multiple possible types
type JSON struct {
valuesByPath map[string]any
}

// NewJSON creates a new empty JSON value
func NewJSON() *JSON {
return &JSON{
valuesByPath: make(map[string]any),
}
}

func (o *JSON) ValuesByPath() map[string]any {
return o.valuesByPath
}

func (o *JSON) SetValueAtPath(path string, value any) {
o.valuesByPath[path] = value
}

func (o *JSON) ValueAtPath(path string) (any, bool) {
value, ok := o.valuesByPath[path]
return value, ok
}

// MarshalJSON implements the json.Marshaler interface
func (o *JSON) MarshalJSON() ([]byte, error) {
return json.Marshal(o.valuesByPath)
}

// Scan implements the sql.Scanner interface
func (o *JSON) Scan(value interface{}) error {
valuesByPath, ok := value.(map[string]any)
if !ok {
return fmt.Errorf("JSON Scan value must be map[string]any")
}

o.valuesByPath = valuesByPath
return nil
}

// Value implements the driver.Valuer interface
func (o *JSON) Value() (driver.Value, error) {
return o.valuesByPath, nil
}
Loading

0 comments on commit b37eca7

Please sign in to comment.