ClickHouse / clickhouse-go

Golang driver for ClickHouse
Apache License 2.0
2.87k stars 549 forks source link

High memory consumption INSERTing #1384

Open vkazmirchuk opened 3 weeks ago

vkazmirchuk commented 3 weeks ago

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

jkaflik commented 2 weeks ago

@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 commented 2 weeks ago

@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 commented 2 weeks ago

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

jkaflik commented 2 weeks ago

@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 commented 2 weeks ago

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 commented 2 weeks ago

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.