aerc/worker/imap/worker.go

304 lines
7.5 KiB
Go
Raw Normal View History

package imap
import (
2018-01-14 11:30:11 +01:00
"fmt"
"net/url"
"time"
2018-01-10 17:19:45 +01:00
2018-01-14 11:30:11 +01:00
"github.com/emersion/go-imap"
2020-09-12 15:05:02 +02:00
sortthread "github.com/emersion/go-imap-sortthread"
2018-02-01 03:18:21 +01:00
"github.com/emersion/go-imap/client"
"github.com/pkg/errors"
imap: add option to cache headers Add option to cache headers for imap accounts. Cache db is located at $XDG_CACHE_DIR/aerc/{account name}. The cache is cleaned of stale entries when aerc is first opened. Two new account level configuration options are introduced: * cache-headers (Default: false) * cache-max-age (Default: 30 days (720 hours)) The change in worker/imap/open.go is to set the selected directory. This is required to access the UIDVALIDITY field, which is used in combination with the message ID to form the key for use in the cache db. The key structure is: "header.{UIDVALIDITY}.{UID}" Where reasonable, cache does not stop aerc from running. In general, if there is an error in the cache, aerc should continue working as usual. Errors are either displayed to the user or logged. All messages are stored without flags, and when retrieved have the flags set to SEEN. This is to prevent UI flashes. A new method to FetchMessageFlags is introduced to update flags of cached headers. This is done asynchronously, and the user will see their messages appear and then any flags updated. The message will initially show as SEEN, but will update to unread. I considered updating the cache with the last-known flag state, however it seems prudent to spare the R/W cycle and assume that - eventually - all messages will end up read, and if it isn't the update will occur rather quickly. Note that leveldb puts a lock on the database, preventing multiple instances of aerc from accessing the cache at the same time. Much of this work is based on previous efforts by Vladimír Magyar. Implements: https://todo.sr.ht/~rjarry/aerc/2 Thanks: Vladimír Magyar <vladimir@mgyar.me> Signed-off-by: Tim Culverhouse <tim@timculverhouse.com> Tested-by: inwit <inwit@sindominio.net> Reviewed-by: Koni Marti <koni.marti@gmail.com> Acked-by: Robin Jarry <robin@jarry.cc>
2022-06-15 14:23:51 +02:00
"github.com/syndtr/goleveldb/leveldb"
2018-02-01 03:18:21 +01:00
"git.sr.ht/~rjarry/aerc/lib"
"git.sr.ht/~rjarry/aerc/logging"
"git.sr.ht/~rjarry/aerc/models"
"git.sr.ht/~rjarry/aerc/worker/handlers"
"git.sr.ht/~rjarry/aerc/worker/types"
)
func init() {
handlers.RegisterWorkerFactory("imap", NewIMAPWorker)
handlers.RegisterWorkerFactory("imaps", NewIMAPWorker)
}
var (
errUnsupported = fmt.Errorf("unsupported command")
errClientNotReady = fmt.Errorf("client not ready")
errNotConnected = fmt.Errorf("not connected")
errAlreadyConnected = fmt.Errorf("already connected")
)
2018-01-14 11:30:11 +01:00
type imapClient struct {
*client.Client
thread *sortthread.ThreadClient
sort *sortthread.SortClient
2018-01-14 11:30:11 +01:00
}
type imapConfig struct {
scheme string
insecure bool
addr string
user *url.Userinfo
folders []string
oauthBearer lib.OAuthBearer
xoauth2 lib.Xoauth2
idle_timeout time.Duration
idle_debounce time.Duration
reconnect_maxwait time.Duration
// tcp connection parameters
connection_timeout time.Duration
keepalive_period time.Duration
keepalive_probes int
keepalive_interval int
imap: add option to cache headers Add option to cache headers for imap accounts. Cache db is located at $XDG_CACHE_DIR/aerc/{account name}. The cache is cleaned of stale entries when aerc is first opened. Two new account level configuration options are introduced: * cache-headers (Default: false) * cache-max-age (Default: 30 days (720 hours)) The change in worker/imap/open.go is to set the selected directory. This is required to access the UIDVALIDITY field, which is used in combination with the message ID to form the key for use in the cache db. The key structure is: "header.{UIDVALIDITY}.{UID}" Where reasonable, cache does not stop aerc from running. In general, if there is an error in the cache, aerc should continue working as usual. Errors are either displayed to the user or logged. All messages are stored without flags, and when retrieved have the flags set to SEEN. This is to prevent UI flashes. A new method to FetchMessageFlags is introduced to update flags of cached headers. This is done asynchronously, and the user will see their messages appear and then any flags updated. The message will initially show as SEEN, but will update to unread. I considered updating the cache with the last-known flag state, however it seems prudent to spare the R/W cycle and assume that - eventually - all messages will end up read, and if it isn't the update will occur rather quickly. Note that leveldb puts a lock on the database, preventing multiple instances of aerc from accessing the cache at the same time. Much of this work is based on previous efforts by Vladimír Magyar. Implements: https://todo.sr.ht/~rjarry/aerc/2 Thanks: Vladimír Magyar <vladimir@mgyar.me> Signed-off-by: Tim Culverhouse <tim@timculverhouse.com> Tested-by: inwit <inwit@sindominio.net> Reviewed-by: Koni Marti <koni.marti@gmail.com> Acked-by: Robin Jarry <robin@jarry.cc>
2022-06-15 14:23:51 +02:00
cacheEnabled bool
cacheMaxAge time.Duration
}
type IMAPWorker struct {
config imapConfig
2018-01-14 11:30:11 +01:00
client *imapClient
selected *imap.MailboxStatus
updates chan client.Update
worker *types.Worker
seqMap SeqMap
idler *idler
observer *observer
imap: add option to cache headers Add option to cache headers for imap accounts. Cache db is located at $XDG_CACHE_DIR/aerc/{account name}. The cache is cleaned of stale entries when aerc is first opened. Two new account level configuration options are introduced: * cache-headers (Default: false) * cache-max-age (Default: 30 days (720 hours)) The change in worker/imap/open.go is to set the selected directory. This is required to access the UIDVALIDITY field, which is used in combination with the message ID to form the key for use in the cache db. The key structure is: "header.{UIDVALIDITY}.{UID}" Where reasonable, cache does not stop aerc from running. In general, if there is an error in the cache, aerc should continue working as usual. Errors are either displayed to the user or logged. All messages are stored without flags, and when retrieved have the flags set to SEEN. This is to prevent UI flashes. A new method to FetchMessageFlags is introduced to update flags of cached headers. This is done asynchronously, and the user will see their messages appear and then any flags updated. The message will initially show as SEEN, but will update to unread. I considered updating the cache with the last-known flag state, however it seems prudent to spare the R/W cycle and assume that - eventually - all messages will end up read, and if it isn't the update will occur rather quickly. Note that leveldb puts a lock on the database, preventing multiple instances of aerc from accessing the cache at the same time. Much of this work is based on previous efforts by Vladimír Magyar. Implements: https://todo.sr.ht/~rjarry/aerc/2 Thanks: Vladimír Magyar <vladimir@mgyar.me> Signed-off-by: Tim Culverhouse <tim@timculverhouse.com> Tested-by: inwit <inwit@sindominio.net> Reviewed-by: Koni Marti <koni.marti@gmail.com> Acked-by: Robin Jarry <robin@jarry.cc>
2022-06-15 14:23:51 +02:00
cache *leveldb.DB
caps *models.Capabilities
}
func NewIMAPWorker(worker *types.Worker) (types.Backend, error) {
return &IMAPWorker{
2019-05-14 02:16:55 +02:00
updates: make(chan client.Update, 50),
worker: worker,
selected: &imap.MailboxStatus{},
idler: newIdler(imapConfig{}, worker),
observer: newObserver(imapConfig{}, worker),
caps: &models.Capabilities{},
}, nil
}
func (w *IMAPWorker) newClient(c *client.Client) {
c.Updates = w.updates
w.client = &imapClient{c, sortthread.NewThreadClient(c), sortthread.NewSortClient(c)}
w.idler.SetClient(w.client)
w.observer.SetClient(w.client)
sort, err := w.client.sort.SupportSort()
if err == nil && sort {
w.caps.Sort = true
logging.Infof("Server Capability found: Sort")
}
thread, err := w.client.thread.SupportThread()
if err == nil && thread {
w.caps.Thread = true
logging.Infof("Server Capability found: Thread")
}
}
2018-01-14 11:30:11 +01:00
func (w *IMAPWorker) handleMessage(msg types.WorkerMessage) error {
defer func() {
w.idler.Start()
}()
if err := w.idler.Stop(); err != nil {
return err
}
var reterr error // will be returned at the end, needed to support idle
2019-05-14 02:16:55 +02:00
// when client is nil allow only certain messages to be handled
if w.client == nil {
switch msg.(type) {
case *types.Connect, *types.Reconnect, *types.Disconnect, *types.Configure:
default:
return errClientNotReady
}
}
// set connection timeout for calls to imap server
if w.client != nil {
w.client.Timeout = w.config.connection_timeout
}
2018-01-10 17:19:45 +01:00
switch msg := msg.(type) {
2018-02-02 01:54:19 +01:00
case *types.Unsupported:
2018-01-14 11:30:11 +01:00
// No-op
2018-02-02 01:54:19 +01:00
case *types.Configure:
reterr = w.handleConfigure(msg)
2018-02-02 01:54:19 +01:00
case *types.Connect:
if w.client != nil && w.client.State() == imap.SelectedState {
if !w.observer.AutoReconnect() {
w.observer.SetAutoReconnect(true)
w.observer.EmitIfNotConnected()
}
reterr = errAlreadyConnected
break
}
w.observer.SetAutoReconnect(true)
c, err := w.connect()
if err != nil {
w.observer.EmitIfNotConnected()
reterr = err
break
2018-01-14 11:30:11 +01:00
}
w.newClient(c)
w.worker.PostMessage(&types.Done{Message: types.RespondTo(msg)}, nil)
case *types.Reconnect:
if !w.observer.AutoReconnect() {
reterr = fmt.Errorf("auto-reconnect is disabled; run connect to enable it")
break
}
c, err := w.connect()
if err != nil {
errReconnect := w.observer.DelayedReconnect()
reterr = errors.Wrap(errReconnect, err.Error())
break
}
w.newClient(c)
w.worker.PostMessage(&types.Done{Message: types.RespondTo(msg)}, nil)
case *types.Disconnect:
w.observer.SetAutoReconnect(false)
w.observer.Stop()
if w.client == nil || w.client.State() != imap.SelectedState {
reterr = errNotConnected
break
}
if err := w.client.Logout(); err != nil {
reterr = err
break
}
w.worker.PostMessage(&types.Done{Message: types.RespondTo(msg)}, nil)
2018-02-02 01:54:19 +01:00
case *types.ListDirectories:
2018-02-02 01:34:08 +01:00
w.handleListDirectories(msg)
2019-01-13 22:18:10 +01:00
case *types.OpenDirectory:
w.handleOpenDirectory(msg)
case *types.FetchDirectoryContents:
w.handleFetchDirectoryContents(msg)
case *types.FetchDirectoryThreaded:
w.handleDirectoryThreaded(msg)
case *types.CreateDirectory:
w.handleCreateDirectory(msg)
case *types.RemoveDirectory:
w.handleRemoveDirectory(msg)
case *types.FetchMessageHeaders:
w.handleFetchMessageHeaders(msg)
2019-03-31 18:14:37 +02:00
case *types.FetchMessageBodyPart:
w.handleFetchMessageBodyPart(msg)
case *types.FetchFullMessages:
w.handleFetchFullMessages(msg)
imap: add option to cache headers Add option to cache headers for imap accounts. Cache db is located at $XDG_CACHE_DIR/aerc/{account name}. The cache is cleaned of stale entries when aerc is first opened. Two new account level configuration options are introduced: * cache-headers (Default: false) * cache-max-age (Default: 30 days (720 hours)) The change in worker/imap/open.go is to set the selected directory. This is required to access the UIDVALIDITY field, which is used in combination with the message ID to form the key for use in the cache db. The key structure is: "header.{UIDVALIDITY}.{UID}" Where reasonable, cache does not stop aerc from running. In general, if there is an error in the cache, aerc should continue working as usual. Errors are either displayed to the user or logged. All messages are stored without flags, and when retrieved have the flags set to SEEN. This is to prevent UI flashes. A new method to FetchMessageFlags is introduced to update flags of cached headers. This is done asynchronously, and the user will see their messages appear and then any flags updated. The message will initially show as SEEN, but will update to unread. I considered updating the cache with the last-known flag state, however it seems prudent to spare the R/W cycle and assume that - eventually - all messages will end up read, and if it isn't the update will occur rather quickly. Note that leveldb puts a lock on the database, preventing multiple instances of aerc from accessing the cache at the same time. Much of this work is based on previous efforts by Vladimír Magyar. Implements: https://todo.sr.ht/~rjarry/aerc/2 Thanks: Vladimír Magyar <vladimir@mgyar.me> Signed-off-by: Tim Culverhouse <tim@timculverhouse.com> Tested-by: inwit <inwit@sindominio.net> Reviewed-by: Koni Marti <koni.marti@gmail.com> Acked-by: Robin Jarry <robin@jarry.cc>
2022-06-15 14:23:51 +02:00
case *types.FetchMessageFlags:
w.handleFetchMessageFlags(msg)
2019-03-21 04:23:38 +01:00
case *types.DeleteMessages:
w.handleDeleteMessages(msg)
case *types.FlagMessages:
w.handleFlagMessages(msg)
2020-05-25 16:59:48 +02:00
case *types.AnsweredMessages:
w.handleAnsweredMessages(msg)
2019-05-14 22:34:42 +02:00
case *types.CopyMessages:
w.handleCopyMessages(msg)
case *types.MoveMessages:
w.handleMoveMessages(msg)
case *types.AppendMessage:
w.handleAppendMessage(msg)
case *types.SearchDirectory:
w.handleSearchDirectory(msg)
case *types.CheckMail:
w.handleCheckMailMessage(msg)
2018-01-14 11:30:11 +01:00
default:
reterr = errUnsupported
}
2019-05-14 02:16:55 +02:00
// we don't want idle to timeout, so set timeout to zero
if w.client != nil {
w.client.Timeout = 0
}
return reterr
}
2019-01-13 22:18:10 +01:00
func (w *IMAPWorker) handleImapUpdate(update client.Update) {
logging.Debugf("(= %T", update)
2019-01-13 22:18:10 +01:00
switch update := update.(type) {
case *client.MailboxUpdate:
status := update.Mailbox
if w.selected.Name == status.Name {
w.selected = status
}
2019-01-13 22:18:10 +01:00
w.worker.PostMessage(&types.DirectoryInfo{
Info: &models.DirectoryInfo{
Flags: status.Flags,
Name: status.Name,
ReadOnly: status.ReadOnly,
Exists: int(status.Messages),
Recent: int(status.Recent),
Unseen: int(status.Unseen),
Caps: w.caps,
},
2019-01-13 22:18:10 +01:00
}, nil)
case *client.MessageUpdate:
msg := update.Message
if msg.Uid == 0 {
if uid, found := w.seqMap.Get(msg.SeqNum); !found {
logging.Errorf("MessageUpdate unknown seqnum: %d", msg.SeqNum)
return
} else {
msg.Uid = uid
}
}
seqmap: refactor seqmap to use slice instead of map The imap worker's seqmap is represented as a map of sequence number to UID. This presents a problem when expunging group of messages from the mailbox: each individual expunge decrements the sequence numbers by 1 (for every sequence number greater than the expunged). This requires a looping around the map to update the keys. The use of a map also requires that both the sequence number and the UID of a message be known in order to insert it into the map. This is only discovered by fetching individual message body parts (flags, headers, etc), leaving the seqmap to be empty until we have fetched information about each message. In certain instances (if a mailbox has recently been loaded), all information is loaded in memory and no new information is fetched - leaving the seqmap empty and the UI out of sync with the worker. Refactor the seqmap as a slice, so that any expunge automatically decrements the rest of the sequences. Use the results of FetchDirectoryContents or FetchDirectoryThreaded to initialize the seqmap with all discovered UIDs. Sort the UIDs in ascending order: IMAP specification requires that sequence numbers start at 1 increase in order of ascending UID. Add individual messages to the map if they come via a MessageUpdate and have a sequence number larger than our slice. Update seqmap tests with new logic. Reference: https://datatracker.ietf.org/doc/html/rfc3501#section-2.3.1.2 Fixes: https://todo.sr.ht/~rjarry/aerc/69 Signed-off-by: Tim Culverhouse <tim@timculverhouse.com> Acked-by: Robin Jarry <robin@jarry.cc>
2022-08-01 19:18:25 +02:00
if int(msg.SeqNum) > w.seqMap.Size() {
w.seqMap.Put(msg.Uid)
}
w.worker.PostMessage(&types.MessageInfo{
Info: &models.MessageInfo{
BodyStructure: translateBodyStructure(msg.BodyStructure),
Envelope: translateEnvelope(msg.Envelope),
Flags: translateImapFlags(msg.Flags),
InternalDate: msg.InternalDate,
Uid: msg.Uid,
},
}, nil)
2019-05-14 02:23:23 +02:00
case *client.ExpungeUpdate:
if uid, found := w.seqMap.Pop(update.SeqNum); !found {
logging.Errorf("ExpungeUpdate unknown seqnum: %d", update.SeqNum)
} else {
w.worker.PostMessage(&types.MessagesDeleted{
Uids: []uint32{uid},
}, nil)
}
2019-01-13 22:18:10 +01:00
}
}
func (w *IMAPWorker) Run() {
for {
select {
2018-02-02 00:42:03 +01:00
case msg := <-w.worker.Actions:
msg = w.worker.ProcessAction(msg)
if err := w.handleMessage(msg); errors.Is(err, errUnsupported) {
2018-02-02 01:54:19 +01:00
w.worker.PostMessage(&types.Unsupported{
2018-01-14 11:30:11 +01:00
Message: types.RespondTo(msg),
2018-02-02 00:42:03 +01:00
}, nil)
2018-01-14 11:30:11 +01:00
} else if err != nil {
2018-02-02 01:54:19 +01:00
w.worker.PostMessage(&types.Error{
2018-01-14 11:30:11 +01:00
Message: types.RespondTo(msg),
Error: err,
2018-02-02 00:42:03 +01:00
}, nil)
2018-01-14 11:30:11 +01:00
}
2018-01-14 11:30:11 +01:00
case update := <-w.updates:
2019-01-13 22:18:10 +01:00
w.handleImapUpdate(update)
}
}
}