Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

High memory consumption INSERTing #1384

Open
7 tasks done
vkazmirchuk opened this issue Aug 23, 2024 · 11 comments
Open
7 tasks done

High memory consumption INSERTing #1384

vkazmirchuk opened this issue Aug 23, 2024 · 11 comments

Comments

@vkazmirchuk
Copy link

vkazmirchuk commented Aug 23, 2024

Observed

We insert 1 million records at a time using the function batch.AppendStruct(item)
After some number of iterations we had high memory consumption by the clickhouse client.

Type: inuse_space
Time: Aug 22, 2024 at 11:40am (+04)
Entering interactive mode (type "help" for commands, "o" for options)
(pprof) top
Showing nodes accounting for 20049.33MB, 97.87% of 20485.24MB total
Dropped 144 nodes (cum <= 102.43MB)
Showing top 10 nodes out of 31
      flat  flat%   sum%        cum   cum%
 7554.70MB 36.88% 36.88%  7554.70MB 36.88%  github.com/ClickHouse/ch-go/compress.(*Writer).Compress
 6764.72MB 33.02% 69.90%  6764.72MB 33.02%  github.com/ClickHouse/ch-go/proto.ColStr.EncodeColumn
 2455.30MB 11.99% 81.89%  2458.01MB 12.00%  github.com/bytedance/sonic.frozenConfig.Unmarshal
 1498.78MB  7.32% 89.20%  1498.78MB  7.32%  github.com/G-Core/cdn-analytics-platform/src/log-sender/internal/domain.init.func6
 1362.98MB  6.65% 95.86%  1362.98MB  6.65%  github.com/ClickHouse/ch-go/proto.(*ColStr).Append (inline)
  184.73MB   0.9% 96.76%   184.73MB   0.9%  github.com/G-Core/cdn-analytics-platform/src/log-sender/internal/accumulator.NewAccumulator
  182.13MB  0.89% 97.65%   223.74MB  1.09%  github.com/ClickHouse/clickhouse-go/v2/lib/column.(*LowCardinality).AppendRow
(pprof) list Compress
Total: 20.01GB
ROUTINE ======================== github.com/ClickHouse/ch-go/compress.(*Writer).Compress in cdn-analytics-platform/vendor/github.com/ClickHouse/ch-go/compress/writer.go
    7.38GB     7.38GB (flat, cum) 36.88% of Total
         .          .     21:func (w *Writer) Compress(m Method, buf []byte) error {
         .          .     22:	maxSize := lz4.CompressBlockBound(len(buf))
    7.38GB     7.38GB     23:	w.Data = append(w.Data[:0], make([]byte, maxSize+headerSize)...)
         .          .     24:	_ = w.Data[:headerSize]
         .          .     25:	w.Data[hMethod] = byte(m)
         .          .     26:
         .          .     27:	var n int
         .          .     28:
(pprof) list proto.ColStr.EncodeColumn
Total: 20.01GB
ROUTINE ======================== github.com/ClickHouse/ch-go/proto.ColStr.EncodeColumn in cdn-analytics-platform/vendor/github.com/ClickHouse/ch-go/proto/col_str.go
    6.61GB     6.61GB (flat, cum) 33.02% of Total
         .          .     70:func (c ColStr) EncodeColumn(b *Buffer) {
         .          .     71:	buf := make([]byte, binary.MaxVarintLen64)
         .          .     72:	for _, p := range c.Pos {
         .          .     73:		n := binary.PutUvarint(buf, uint64(p.End-p.Start))
         .          .     74:		b.Buf = append(b.Buf, buf[:n]...)
    6.61GB     6.61GB     75:		b.Buf = append(b.Buf, c.Buf[p.Start:p.End]...)
         .          .     76:	}
         .          .     77:}
         .          .     78:
         .          .     79:// ForEach calls f on each string from column.
         .          .     80:func (c ColStr) ForEach(f func(i int, s string) error) error {

pprof memory report: pprof.alloc_objects.alloc_space.inuse_objects.inuse_space.028.pb.gz

Our golang structure that we put into the database:

type Item struct {
	FieldA       time.Time `ch:"field_a"`
	FieldB       time.Time `ch:"field_b"`
	FieldC       net.IP    `ch:"field_c"`
	FieldD       string    `ch:"field_d"`
	FieldE       string    `ch:"field_e"`
	FieldF       string    `ch:"field_f"`
	FieldG       string    `ch:"field_g"`
	FieldH       string    `ch:"field_h"`
	FieldI       uint16    `ch:"field_i"`
	FieldJ       int64     `ch:"field_j"`
	FieldK       string    `ch:"field_k"`
	FieldL       string    `ch:"field_l"`
	FieldM       int64     `ch:"field_m"`
	FieldN       string    `ch:"field_n"`
	FieldO       uint32    `ch:"field_o"`
	FieldP       string    `ch:"field_p"`
	FieldQ       []uint32  `ch:"field_q"`
	FieldR       []int64   `ch:"field_r"`
	FieldS       string    `ch:"field_s"`
	FieldT       []uint16  `ch:"field_t"`
	FieldU       []uint32  `ch:"field_u"`
	FieldV       []uint32  `ch:"field_v"`
	FieldW       int32     `ch:"field_w"`
	FieldX       int32     `ch:"field_x"`
	FieldY       string    `ch:"field_y"`
	FieldZ       net.IP    `ch:"field_z"`
	FieldAA      string    `ch:"field_aa"`
	FieldAB      string    `ch:"field_ab"`
	FieldAC      string    `ch:"field_ac"`
	FieldAD      uint32    `ch:"field_ad"`
	FieldAE      string    `ch:"field_ae"`
	FieldAF      string    `ch:"field_af"`
	FieldAG      string    `ch:"field_ag"`
	FieldAH      string    `ch:"field_ah"`
	FieldAI      string    `ch:"field_ai"`
	FieldAJ      string    `ch:"field_aj"`
	FieldAK      string    `ch:"field_ak"`
	FieldAL      string    `ch:"field_al"`
	FieldAM      string    `ch:"field_am"`
	FieldAN      string    `ch:"field_an"`
	FieldAO      uint8     `ch:"field_ao"`
	FieldAP      string    `ch:"field_ap"`
	FieldAQ      []net.IP  `ch:"field_aq"`
	FieldAR      uint64    `ch:"field_ar"`
	FieldAS      string    `ch:"field_as"`
	FieldAT      uint32    `ch:"field_at"`
	FieldAU      uint32    `ch:"field_au"`
	FieldAV      string    `ch:"field_av"`
	FieldAW      uint16    `ch:"field_aw"`
	FieldAX      uint16    `ch:"field_ax"`
	FieldAY      int8      `ch:"field_ay"`
	FieldAZ      string    `ch:"field_az"`
}

Expected behaviour

The client should reuse memory whenever possible, rather than allocating new memory at each iteration of batch insertion

Code example

query := "INSERT INTO target_table"

batch, err := conn.PrepareBatch(ctx, query)
if err != nil {
    return fmt.Errorf("prepare batch: %v", err)
}

for _, item := range items {
    if err := batch.AppendStruct(item); err != nil {
        return fmt.Errorf("append to batch: %v", err)
    }
}

if err := batch.Send(); err != nil {
    return fmt.Errorf("send batch: %v", err)
}
        

Details

Environment

  • clickhouse-go version: v2.25.0
  • Interface: ClickHouse API / database/sql compatible driver: ClickHouse API
  • Go version: 1.22.1
  • Operating system: Linux
  • ClickHouse version: 23.8.8.20
  • Is it a ClickHouse Cloud? No
  • CREATE TABLE statements for tables involved:
CREATE TABLE target_table (
    `field_a` DateTime('UTC'),
    `field_b` Date,
    `field_c` IPv6,
    `field_d` LowCardinality(String),
    `field_e` String,
    `field_f` String,
    `field_g` LowCardinality(String),
    `field_h` LowCardinality(String),
    `field_i` UInt16,
    `field_j` Int64,
    `field_k` String,
    `field_l` String,
    `field_m` Int64,
    `field_n` String,
    `field_o` UInt32,
    `field_p` LowCardinality(String),
    `field_q` Array(UInt32),
    `field_r` Array(Int64),
    `field_s` String,
    `field_t` Array(UInt16),
    `field_u` Array(UInt32),
    `field_v` Array(UInt32),
    `field_w` Int32,
    `field_x` Int32,
    `field_y` LowCardinality(String),
    `field_z` IPv6,
    `field_aa` String,
    `field_ab` LowCardinality(String),
    `field_ac` LowCardinality(String),
    `field_ad` Nullable(UInt32),
    `field_ae` LowCardinality(String),
    `field_af` LowCardinality(String),
    `field_ag` String,
    `field_ah` LowCardinality(String),
    `field_ai` LowCardinality(String),
    `field_aj` LowCardinality(String),
    `field_ak` LowCardinality(String),
    `field_al` LowCardinality(String),
    `field_am` LowCardinality(String),
    `field_an` LowCardinality(String),
    `field_ao` UInt8,
    `field_ap` LowCardinality(String),
    `field_aq` Array(IPv6),
    `field_ar` UInt64,
    `field_as` LowCardinality(String),
    `field_at` UInt32,
    `field_au` UInt32,
    `field_av` LowCardinality(String),
    `field_aw` Nullable(UInt16),
    `field_ax` Nullable(UInt32),
    `field_ay` Int8,
    `field_az` LowCardinality(String)
) 
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/table', '{replica}')
PARTITION BY toYYYYMMDD(day)
ORDER BY (field_w, field_x, field_ab, field_av, field_ar)
TTL field_b + toIntervalDay(4)
@jkaflik
Copy link
Contributor

jkaflik commented Aug 28, 2024

@vkazmirchuk thanks for reporting this.

Do I understand correctly memory consumption grows over time?

After some number of iterations we had high memory consumption by the clickhouse client.

Do you have any runtime statistics on GC attempting to free memory? Is memory going to the OS/container limit?

@vkazmirchuk
Copy link
Author

vkazmirchuk commented Aug 30, 2024

@jkaflik

Do I understand correctly memory consumption grows over time?

Yes, it happens after a lot of baches insertions

Do you have any runtime statistics on GC attempting to free memory? Is memory going to the OS/container limit?

We tried to run GC manually after each insertion, but it doesn't release memory completely and for a long time the client accumulates memory which is not cleared.

@jkaflik
Copy link
Contributor

jkaflik commented Aug 30, 2024

@vkazmirchuk Could you also check for number of goroutines? runtime.NumGoroutine() Do you run any of INSERT in goroutine?

@jkaflik
Copy link
Contributor

jkaflik commented Aug 30, 2024

@vkazmirchuk, just to clarify - because GC in Go and memory management in general is tricky. Let me ask again:

Is memory going to the OS/container limit?

do you encounter out-of-memory? I want to double-check if it's GC not releasing or we have problem with a stuff not releasing a memory.

See: https://pkg.go.dev/runtime/debug#FreeOSMemory (https://stackoverflow.com/questions/37382600/cannot-free-memory-once-occupied-by-bytes-buffer/37383604#37383604)

If you are not hitting out-of-memory, you could play with GOMEMLIMIT to override container/host memory info: https://tip.golang.org/doc/gc-guide#Memory_limit

@vkazmirchuk
Copy link
Author

Could you also check for number of goroutines? runtime.NumGoroutine() Do you run any of INSERT in goroutine?

We have one gorutine that performs insertions of 1 bach one after another. The total number of gorutines in the appendix is 23. But I don't think that plays any role.

do you encounter out-of-memory? I want to double-check if it's GC not releasing or we have problem with a stuff not releasing a memory.

Yeah, we end up catching the OOM.

If you are not hitting out-of-memory, you could play with GOMEMLIMIT to override container/host memory info

This is what helped us not to catch OOM, we limit memory to 40 gigabytes and it saves us, but our application after a while eats all 40 gigabytes and stays at that level.

It would be ideal to reuse memory within the client to clickhouse. We have implemented sync.Pool in our application for many things and it has helped us a lot in optimisation.

@jkaflik
Copy link
Contributor

jkaflik commented Aug 30, 2024

we limit memory to 40 gigabytes and it saves us, but our application after a while eats all 40 gigabytes and stays at that level.

When you set GOMEMLIMIT to 40 GiB, it stays at this level, and it does not go over the limit, right? It sounds like GC is not freeing memory to the OS. This is something that can happen.

What if you lower GOMEMLIMIT? This can influence CPU cycles as GC will be executed more often.


Besides that, of course, we should invest a bit into figuring out how we can save us from unnecessary memory allocations.

Current driver architecture assumes everything is buffered per each block. Thus, high memory consumption can happen. For now what I can recommend is to lower batch size.

@vkazmirchuk
Copy link
Author

vkazmirchuk commented Sep 17, 2024

When you set GOMEMLIMIT to 40 GiB, it stays at this level, and it does not go over the limit, right? It sounds like GC is not freeing memory to the OS. This is something that can happen.

Yes over 40 GiB is not out, but for us even that value is great. Because 1 batch takes about 600 MiB maximum.

What if you lower GOMEMLIMIT? This can influence CPU cycles as GC will be executed more often.

That's right, that's what happens. But the point is that we are trying to achieve good performance with low CPU and RAM consumption. We overuse memory almost everywhere in our application so as not to strain GC. And the only weak point in the app is the clickhouse client

Current driver architecture assumes everything is buffered per each block. Thus, high memory consumption can happen. For now what I can recommend is to lower batch size.

We have a fairly large clichhouse cluster and the batch size was chosen so that our cluster could handle the load. By reducing the batch size we will need to invest a lot of money in horizontal scaling of the cluster. In general we can do this, but it is a last resort, first we want to solve the problem with the clickhouse client we believe that it can work more efficiently with memory

@jkaflik
Copy link
Contributor

jkaflik commented Sep 17, 2024

Thanks for your comment @vkazmirchuk . We will need to prioritize it on our side.

If you don't need a database/sql nor row-oriented API for your ingestion, I can recommend taking a look at https://github.com/ClickHouse/ch-go. It has less abstraction and gives you better control over data transformations. It can help a bit. Unfortunately, native format transformation in clickhouse-go comes from ch-go, so likely you hit the same allocations problem.

@vkazmirchuk
Copy link
Author

Yes but as you can see in the description of this issue the problem is in this package ch-go =)

@jkaflik
Copy link
Contributor

jkaflik commented Sep 19, 2024

@jkaflik
Copy link
Contributor

jkaflik commented Sep 20, 2024

For ch-go there is a ongoing draft PR that aims to reduce memory overhead: ClickHouse/ch-go#413

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants